commit 32be831ce56beab404d463cd7ada54a98f9e99f8 Author: stack Date: Fri Sep 30 14:54:58 2016 -0700 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. 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 { 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()); 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(); } 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(); } 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(); } 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(); 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(); } 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 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(); 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(); } 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()); 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(); } 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(); } 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); 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); 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 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 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> family: scan.getFamilyMap().entrySet()) { - columnBuilder.setFamily(ByteStringer.wrap(family.getKey())); + columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey())); NavigableSet 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 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 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> families = get.getFamilyMap(); for (Map.Entry> family: families.entrySet()) { NavigableSet 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> family: increment.getFamilyCellMap().entrySet()) { - columnBuilder.setFamily(ByteStringer.wrap(family.getKey())); + columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey())); columnBuilder.clearQualifierValue(); List 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 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> 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 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> 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> 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> 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> 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 tableNamesList) { @@ -2484,7 +2484,7 @@ public final class ProtobufUtil { for (Map.Entry> 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 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 e : hcd.getConfiguration().entrySet()) { @@ -2742,8 +2742,8 @@ public final class ProtobufUtil { builder.setTableName(toProtoTableName(htd.getTableName())); for (Map.Entry 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(); } 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(familyPaths.size()); for(Pair 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(); } /** 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 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(); 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(); } 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(); 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. diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml index 1bd0aa6..e0e9eec 100644 --- a/hbase-protocol-shaded/pom.xml +++ b/hbase-protocol-shaded/pom.xml @@ -1,6 +1,6 @@ - - 4.0.0 - - hbase - org.apache.hbase - 2.0.0-SNAPSHOT - .. - - - hbase-protocol-shaded - Apache HBase - Shaded Protocol - Shaded protobuf protocol classes used by HBase internally. - - - true - - 3.1.0 - - - - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.3 - - - package - - shade - - - - - com.google.protobuf - org.apache.hadoop.hbase.shaded.com.google.protobuf - - - - - - commons-logging:commons-logging - com.github.stephenc.findbugs:findbugs-annotations - log4j:log4j - org.hamcrest:hamcrest-core - org.mockito:mockito-all - junit:junit - org.apache.hbase:hbase-annotations - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - 2.10 - - - unpack - package - - unpack - - - - - org.apache.hbase - hbase-protocol-shaded - ${project.version} - jar - true - ${project.build.directory}/classes - - - - - - - - org.apache.maven.plugins - maven-site-plugin - - true - - - - - org.apache.maven.plugins - maven-source-plugin - + 3.1.0 + + ${basedir}/target + + src/main/java + + + + ${sources.dir} + ${classes.dir} + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + org.apache.maven.plugins + maven-source-plugin + maven-assembly-plugin @@ -133,163 +65,283 @@ true + + maven-surefire-plugin + + + + secondPartTestsExecution + test + + test + + + true + + + + + + + + - maven-surefire-plugin - - - - secondPartTestsExecution - test + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.hadoop + hadoop-maven-plugins + [2.0.5-alpha,) - test + protoc - - true - - - + + + + + + + + - - + + + + + + + + org.apache.hbase + hbase-annotations + + + jdk.tools + jdk.tools + + + + + + com.google.protobuf + protobuf-java + ${internal.protobuf.version} + + + commons-logging + commons-logging + + + + + + skip-protocol-shaded-tests + + + skip-protocol-shaded-tests + + + + true + + + + + generate-shaded-classes + + + generate-shaded-classes + + + + generate-shaded-classes + ${project.build.directory}/protoc-generated-sources + ${project.build.directory}/protoc-generated-classes + + ${profile.id}.${artifactId}-${project.version} + + - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 + org.apache.hadoop + hadoop-maven-plugins - - - - - org.apache.hadoop - hadoop-maven-plugins - [2.0.5-alpha,) - - protoc - - - - - - - - + ${internal.protobuf.version} + + + compile-protoc + generate-sources + + protoc + + + + ${basedir}/src/main/protobuf + + + ${basedir}/src/main/protobuf + + + Admin.proto + Cell.proto + Client.proto + ClusterId.proto + ClusterStatus.proto + Comparator.proto + Encryption.proto + ErrorHandling.proto + FS.proto + Filter.proto + HBase.proto + HFile.proto + LoadBalancer.proto + MapReduce.proto + Master.proto + MasterProcedure.proto + Procedure.proto + Quota.proto + RPC.proto + RegionNormalizer.proto + RegionServerStatus.proto + Snapshot.proto + Tracing.proto + WAL.proto + ZooKeeper.proto + TestProcedure.proto + test.proto + test_rpc_service.proto + + + ${sources.dir} + + + - - - - - - - - - org.apache.hbase - hbase-annotations - - - jdk.tools - jdk.tools - - - - - - com.google.protobuf - protobuf-java - ${internal.protobuf.version} - - - commons-logging - commons-logging - - + + org.apache.maven.plugins + maven-jar-plugin + 3.0.2 + + ${jar.finalName} + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.3 + + + package + + shade + + + true + true + + + com.google.protobuf + org.apache.hadoop.hbase.shaded.com.google.protobuf + + + - - skip-protocol-shaded-tests - - - skip-protocol-shaded-tests - - - - true - - - - compile-protobuf - - - compile-protobuf - - - - - - org.apache.hadoop - hadoop-maven-plugins + The list below must exlude all of the above except protobuf. + --> + + + commons-logging:commons-logging + com.github.stephenc.findbugs:findbugs-annotations + log4j:log4j + org.hamcrest:hamcrest-core + org.mockito:mockito-all + junit:junit + org.apache.hbase:hbase-annotations + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.10 + + + unpack + package + + unpack + - ${internal.protobuf.version} + + + ${project.groupId} + ${project.artifactId} + ${project.version} + sources + jar + true + ${default.sources.dir} + **/*.java + + - - - compile-protoc - generate-sources - - protoc - - - - ${basedir}/src/main/protobuf - - - ${basedir}/src/main/protobuf - - - Admin.proto - Cell.proto - Client.proto - ClusterId.proto - ClusterStatus.proto - Comparator.proto - Encryption.proto - ErrorHandling.proto - FS.proto - Filter.proto - HBase.proto - HFile.proto - LoadBalancer.proto - MapReduce.proto - Master.proto - MasterProcedure.proto - Procedure.proto - Quota.proto - RPC.proto - RegionNormalizer.proto - RegionServerStatus.proto - Snapshot.proto - Tracing.proto - WAL.proto - ZooKeeper.proto - TestProcedure.proto - test.proto - test_rpc_service.proto - - - - ${basedir}/src/main/java/ - - - - - - - - + + + + + + + + diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java index 641bde3..2e78905 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java @@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.ipc.protobuf.generated; public final class TestProcedureProtos { private TestProcedureProtos() {} public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) { } public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) { registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry); } public interface TestTableDDLStateDataOrBuilder extends // @@protoc_insertion_point(interface_extends:TestTableDDLStateData) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { /** * required string table_name = 1; @@ -29,18 +29,18 @@ public final class TestProcedureProtos { /** * required string table_name = 1; */ - com.google.protobuf.ByteString + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableNameBytes(); } /** * Protobuf type {@code TestTableDDLStateData} */ public static final class TestTableDDLStateData extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:TestTableDDLStateData) TestTableDDLStateDataOrBuilder { // Use TestTableDDLStateData.newBuilder() to construct. - private TestTableDDLStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private TestTableDDLStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private TestTableDDLStateData() { @@ -48,18 +48,18 @@ public final class TestProcedureProtos { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private TestTableDDLStateData( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -76,29 +76,29 @@ public final class TestProcedureProtos { break; } case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); bitField0_ |= 0x00000001; tableName_ = bs; break; } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -122,8 +122,8 @@ public final class TestProcedureProtos { if (ref instanceof java.lang.String) { return (java.lang.String) ref; } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { tableName_ = s; @@ -134,17 +134,17 @@ public final class TestProcedureProtos { /** * required string table_name = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableNameBytes() { java.lang.Object ref = tableName_; if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); tableName_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } @@ -162,10 +162,10 @@ public final class TestProcedureProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { if (((bitField0_ & 0x00000001) == 0x00000001)) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 1, tableName_); + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, tableName_); } unknownFields.writeTo(output); } @@ -176,7 +176,7 @@ public final class TestProcedureProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, tableName_); + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, tableName_); } size += unknownFields.getSerializedSize(); memoizedSize = size; @@ -221,61 +221,61 @@ public final class TestProcedureProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -293,7 +293,7 @@ public final class TestProcedureProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -301,15 +301,15 @@ public final class TestProcedureProtos { * Protobuf type {@code TestTableDDLStateData} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:TestTableDDLStateData) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateDataOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -322,12 +322,12 @@ public final class TestProcedureProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -338,7 +338,7 @@ public final class TestProcedureProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_descriptor; } @@ -372,29 +372,29 @@ public final class TestProcedureProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData)other); } else { @@ -423,13 +423,13 @@ public final class TestProcedureProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -454,8 +454,8 @@ public final class TestProcedureProtos { public java.lang.String getTableName() { java.lang.Object ref = tableName_; if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { tableName_ = s; @@ -468,17 +468,17 @@ public final class TestProcedureProtos { /** * required string table_name = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableNameBytes() { java.lang.Object ref = tableName_; if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); tableName_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } /** @@ -507,7 +507,7 @@ public final class TestProcedureProtos { * required string table_name = 1; */ public Builder setTableNameBytes( - com.google.protobuf.ByteString value) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } @@ -517,12 +517,12 @@ public final class TestProcedureProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -540,22 +540,22 @@ public final class TestProcedureProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public TestTableDDLStateData parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new TestTableDDLStateData(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -565,17 +565,17 @@ public final class TestProcedureProtos { } - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_TestTableDDLStateData_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_TestTableDDLStateData_fieldAccessorTable; - public static com.google.protobuf.Descriptors.FileDescriptor + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { return descriptor; } - private static com.google.protobuf.Descriptors.FileDescriptor + private static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor descriptor; static { java.lang.String[] descriptorData = { @@ -584,22 +584,22 @@ public final class TestProcedureProtos { ".hadoop.hbase.shaded.ipc.protobuf.genera" + "tedB\023TestProcedureProtos\210\001\001" }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; return null; } }; - com.google.protobuf.Descriptors.FileDescriptor + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] { }, assigner); internal_static_TestTableDDLStateData_descriptor = getDescriptor().getMessageTypes().get(0); internal_static_TestTableDDLStateData_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_TestTableDDLStateData_descriptor, new java.lang.String[] { "TableName", }); } diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java index 0fdd1fb..36a15c9 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java @@ -6,44 +6,44 @@ package org.apache.hadoop.hbase.shaded.ipc.protobuf.generated; public final class TestProtos { private TestProtos() {} public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) { } public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) { registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry); } public interface EmptyRequestProtoOrBuilder extends // @@protoc_insertion_point(interface_extends:EmptyRequestProto) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { } /** * Protobuf type {@code EmptyRequestProto} */ public static final class EmptyRequestProto extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:EmptyRequestProto) EmptyRequestProtoOrBuilder { // Use EmptyRequestProto.newBuilder() to construct. - private EmptyRequestProto(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private EmptyRequestProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private EmptyRequestProto() { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private EmptyRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -61,22 +61,22 @@ public final class TestProtos { } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -93,7 +93,7 @@ public final class TestProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { unknownFields.writeTo(output); } @@ -137,61 +137,61 @@ public final class TestProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -209,7 +209,7 @@ public final class TestProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -217,15 +217,15 @@ public final class TestProtos { * Protobuf type {@code EmptyRequestProto} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:EmptyRequestProto) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProtoOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -238,12 +238,12 @@ public final class TestProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -252,7 +252,7 @@ public final class TestProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyRequestProto_descriptor; } @@ -279,29 +279,29 @@ public final class TestProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto)other); } else { @@ -322,13 +322,13 @@ public final class TestProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -339,12 +339,12 @@ public final class TestProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -362,22 +362,22 @@ public final class TestProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public EmptyRequestProto parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new EmptyRequestProto(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -389,34 +389,34 @@ public final class TestProtos { public interface EmptyResponseProtoOrBuilder extends // @@protoc_insertion_point(interface_extends:EmptyResponseProto) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { } /** * Protobuf type {@code EmptyResponseProto} */ public static final class EmptyResponseProto extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:EmptyResponseProto) EmptyResponseProtoOrBuilder { // Use EmptyResponseProto.newBuilder() to construct. - private EmptyResponseProto(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private EmptyResponseProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private EmptyResponseProto() { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private EmptyResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -434,22 +434,22 @@ public final class TestProtos { } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -466,7 +466,7 @@ public final class TestProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { unknownFields.writeTo(output); } @@ -510,61 +510,61 @@ public final class TestProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -582,7 +582,7 @@ public final class TestProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -590,15 +590,15 @@ public final class TestProtos { * Protobuf type {@code EmptyResponseProto} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:EmptyResponseProto) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProtoOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -611,12 +611,12 @@ public final class TestProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -625,7 +625,7 @@ public final class TestProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EmptyResponseProto_descriptor; } @@ -652,29 +652,29 @@ public final class TestProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto)other); } else { @@ -695,13 +695,13 @@ public final class TestProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -712,12 +712,12 @@ public final class TestProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -735,22 +735,22 @@ public final class TestProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public EmptyResponseProto parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new EmptyResponseProto(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -762,7 +762,7 @@ public final class TestProtos { public interface EchoRequestProtoOrBuilder extends // @@protoc_insertion_point(interface_extends:EchoRequestProto) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { /** * required string message = 1; @@ -775,18 +775,18 @@ public final class TestProtos { /** * required string message = 1; */ - com.google.protobuf.ByteString + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMessageBytes(); } /** * Protobuf type {@code EchoRequestProto} */ public static final class EchoRequestProto extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:EchoRequestProto) EchoRequestProtoOrBuilder { // Use EchoRequestProto.newBuilder() to construct. - private EchoRequestProto(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private EchoRequestProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private EchoRequestProto() { @@ -794,18 +794,18 @@ public final class TestProtos { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private EchoRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -822,29 +822,29 @@ public final class TestProtos { break; } case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); bitField0_ |= 0x00000001; message_ = bs; break; } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -868,8 +868,8 @@ public final class TestProtos { if (ref instanceof java.lang.String) { return (java.lang.String) ref; } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { message_ = s; @@ -880,17 +880,17 @@ public final class TestProtos { /** * required string message = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMessageBytes() { java.lang.Object ref = message_; if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); message_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } @@ -908,10 +908,10 @@ public final class TestProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { if (((bitField0_ & 0x00000001) == 0x00000001)) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 1, message_); + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, message_); } unknownFields.writeTo(output); } @@ -922,7 +922,7 @@ public final class TestProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, message_); + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, message_); } size += unknownFields.getSerializedSize(); memoizedSize = size; @@ -967,61 +967,61 @@ public final class TestProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -1039,7 +1039,7 @@ public final class TestProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -1047,15 +1047,15 @@ public final class TestProtos { * Protobuf type {@code EchoRequestProto} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:EchoRequestProto) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProtoOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -1068,12 +1068,12 @@ public final class TestProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -1084,7 +1084,7 @@ public final class TestProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoRequestProto_descriptor; } @@ -1118,29 +1118,29 @@ public final class TestProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto)other); } else { @@ -1169,13 +1169,13 @@ public final class TestProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -1200,8 +1200,8 @@ public final class TestProtos { public java.lang.String getMessage() { java.lang.Object ref = message_; if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { message_ = s; @@ -1214,17 +1214,17 @@ public final class TestProtos { /** * required string message = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMessageBytes() { java.lang.Object ref = message_; if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); message_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } /** @@ -1253,7 +1253,7 @@ public final class TestProtos { * required string message = 1; */ public Builder setMessageBytes( - com.google.protobuf.ByteString value) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } @@ -1263,12 +1263,12 @@ public final class TestProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -1286,22 +1286,22 @@ public final class TestProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public EchoRequestProto parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new EchoRequestProto(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -1313,7 +1313,7 @@ public final class TestProtos { public interface EchoResponseProtoOrBuilder extends // @@protoc_insertion_point(interface_extends:EchoResponseProto) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { /** * required string message = 1; @@ -1326,18 +1326,18 @@ public final class TestProtos { /** * required string message = 1; */ - com.google.protobuf.ByteString + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMessageBytes(); } /** * Protobuf type {@code EchoResponseProto} */ public static final class EchoResponseProto extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:EchoResponseProto) EchoResponseProtoOrBuilder { // Use EchoResponseProto.newBuilder() to construct. - private EchoResponseProto(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private EchoResponseProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private EchoResponseProto() { @@ -1345,18 +1345,18 @@ public final class TestProtos { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private EchoResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -1373,29 +1373,29 @@ public final class TestProtos { break; } case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); bitField0_ |= 0x00000001; message_ = bs; break; } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -1419,8 +1419,8 @@ public final class TestProtos { if (ref instanceof java.lang.String) { return (java.lang.String) ref; } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { message_ = s; @@ -1431,17 +1431,17 @@ public final class TestProtos { /** * required string message = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMessageBytes() { java.lang.Object ref = message_; if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); message_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } @@ -1459,10 +1459,10 @@ public final class TestProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { if (((bitField0_ & 0x00000001) == 0x00000001)) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 1, message_); + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, message_); } unknownFields.writeTo(output); } @@ -1473,7 +1473,7 @@ public final class TestProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, message_); + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, message_); } size += unknownFields.getSerializedSize(); memoizedSize = size; @@ -1518,61 +1518,61 @@ public final class TestProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -1590,7 +1590,7 @@ public final class TestProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -1598,15 +1598,15 @@ public final class TestProtos { * Protobuf type {@code EchoResponseProto} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:EchoResponseProto) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProtoOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -1619,12 +1619,12 @@ public final class TestProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -1635,7 +1635,7 @@ public final class TestProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_EchoResponseProto_descriptor; } @@ -1669,29 +1669,29 @@ public final class TestProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto)other); } else { @@ -1720,13 +1720,13 @@ public final class TestProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -1751,8 +1751,8 @@ public final class TestProtos { public java.lang.String getMessage() { java.lang.Object ref = message_; if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { message_ = s; @@ -1765,17 +1765,17 @@ public final class TestProtos { /** * required string message = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMessageBytes() { java.lang.Object ref = message_; if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); message_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } /** @@ -1804,7 +1804,7 @@ public final class TestProtos { * required string message = 1; */ public Builder setMessageBytes( - com.google.protobuf.ByteString value) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } @@ -1814,12 +1814,12 @@ public final class TestProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -1837,22 +1837,22 @@ public final class TestProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public EchoResponseProto parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new EchoResponseProto(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -1864,7 +1864,7 @@ public final class TestProtos { public interface PauseRequestProtoOrBuilder extends // @@protoc_insertion_point(interface_extends:PauseRequestProto) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { /** * required uint32 ms = 1; @@ -1879,11 +1879,11 @@ public final class TestProtos { * Protobuf type {@code PauseRequestProto} */ public static final class PauseRequestProto extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:PauseRequestProto) PauseRequestProtoOrBuilder { // Use PauseRequestProto.newBuilder() to construct. - private PauseRequestProto(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private PauseRequestProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private PauseRequestProto() { @@ -1891,18 +1891,18 @@ public final class TestProtos { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private PauseRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -1925,22 +1925,22 @@ public final class TestProtos { } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -1977,7 +1977,7 @@ public final class TestProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeUInt32(1, ms_); @@ -1991,7 +1991,7 @@ public final class TestProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream .computeUInt32Size(1, ms_); } size += unknownFields.getSerializedSize(); @@ -2037,61 +2037,61 @@ public final class TestProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -2109,7 +2109,7 @@ public final class TestProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -2117,15 +2117,15 @@ public final class TestProtos { * Protobuf type {@code PauseRequestProto} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:PauseRequestProto) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProtoOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -2138,12 +2138,12 @@ public final class TestProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -2154,7 +2154,7 @@ public final class TestProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_PauseRequestProto_descriptor; } @@ -2188,29 +2188,29 @@ public final class TestProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto)other); } else { @@ -2237,13 +2237,13 @@ public final class TestProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -2287,12 +2287,12 @@ public final class TestProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -2310,22 +2310,22 @@ public final class TestProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public PauseRequestProto parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new PauseRequestProto(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -2337,7 +2337,7 @@ public final class TestProtos { public interface AddrResponseProtoOrBuilder extends // @@protoc_insertion_point(interface_extends:AddrResponseProto) - com.google.protobuf.MessageOrBuilder { + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { /** * required string addr = 1; @@ -2350,18 +2350,18 @@ public final class TestProtos { /** * required string addr = 1; */ - com.google.protobuf.ByteString + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getAddrBytes(); } /** * Protobuf type {@code AddrResponseProto} */ public static final class AddrResponseProto extends - com.google.protobuf.GeneratedMessageV3 implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements // @@protoc_insertion_point(message_implements:AddrResponseProto) AddrResponseProtoOrBuilder { // Use AddrResponseProto.newBuilder() to construct. - private AddrResponseProto(com.google.protobuf.GeneratedMessageV3.Builder builder) { + private AddrResponseProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { super(builder); } private AddrResponseProto() { @@ -2369,18 +2369,18 @@ public final class TestProtos { } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet getUnknownFields() { return this.unknownFields; } private AddrResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { this(); int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); try { boolean done = false; while (!done) { @@ -2397,29 +2397,29 @@ public final class TestProtos { break; } case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); bitField0_ |= 0x00000001; addr_ = bs; break; } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( e).setUnfinishedMessage(this); } finally { this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -2443,8 +2443,8 @@ public final class TestProtos { if (ref instanceof java.lang.String) { return (java.lang.String) ref; } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { addr_ = s; @@ -2455,17 +2455,17 @@ public final class TestProtos { /** * required string addr = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getAddrBytes() { java.lang.Object ref = addr_; if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); addr_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } @@ -2483,10 +2483,10 @@ public final class TestProtos { return true; } - public void writeTo(com.google.protobuf.CodedOutputStream output) + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) throws java.io.IOException { if (((bitField0_ & 0x00000001) == 0x00000001)) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 1, addr_); + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, addr_); } unknownFields.writeTo(output); } @@ -2497,7 +2497,7 @@ public final class TestProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, addr_); + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, addr_); } size += unknownFields.getSerializedSize(); memoizedSize = size; @@ -2542,61 +2542,61 @@ public final class TestProtos { } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseDelimitedFrom( java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseDelimitedWithIOException(PARSER, input, extensionRegistry); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( - com.google.protobuf.CodedInputStream input) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input); } public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - return com.google.protobuf.GeneratedMessageV3 + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .parseWithIOException(PARSER, input, extensionRegistry); } @@ -2614,7 +2614,7 @@ public final class TestProtos { @java.lang.Override protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { Builder builder = new Builder(parent); return builder; } @@ -2622,15 +2622,15 @@ public final class TestProtos { * Protobuf type {@code AddrResponseProto} */ public static final class Builder extends - com.google.protobuf.GeneratedMessageV3.Builder implements + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:AddrResponseProto) org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProtoOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_descriptor; } - protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_fieldAccessorTable .ensureFieldAccessorsInitialized( @@ -2643,12 +2643,12 @@ public final class TestProtos { } private Builder( - com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); maybeForceBuilderInitialization(); } private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 .alwaysUseFieldBuilders) { } } @@ -2659,7 +2659,7 @@ public final class TestProtos { return this; } - public com.google.protobuf.Descriptors.Descriptor + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.internal_static_AddrResponseProto_descriptor; } @@ -2693,29 +2693,29 @@ public final class TestProtos { return (Builder) super.clone(); } public Builder setField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.setField(field, value); } public Builder clearField( - com.google.protobuf.Descriptors.FieldDescriptor field) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { return (Builder) super.clearField(field); } public Builder clearOneof( - com.google.protobuf.Descriptors.OneofDescriptor oneof) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { return (Builder) super.clearOneof(oneof); } public Builder setRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, int index, Object value) { return (Builder) super.setRepeatedField(field, index, value); } public Builder addRepeatedField( - com.google.protobuf.Descriptors.FieldDescriptor field, + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, Object value) { return (Builder) super.addRepeatedField(field, value); } - public Builder mergeFrom(com.google.protobuf.Message other) { + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto) { return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto)other); } else { @@ -2744,13 +2744,13 @@ public final class TestProtos { } public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -2775,8 +2775,8 @@ public final class TestProtos { public java.lang.String getAddr() { java.lang.Object ref = addr_; if (!(ref instanceof java.lang.String)) { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { addr_ = s; @@ -2789,17 +2789,17 @@ public final class TestProtos { /** * required string addr = 1; */ - public com.google.protobuf.ByteString + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getAddrBytes() { java.lang.Object ref = addr_; if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); addr_ = b; return b; } else { - return (com.google.protobuf.ByteString) ref; + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; } } /** @@ -2828,7 +2828,7 @@ public final class TestProtos { * required string addr = 1; */ public Builder setAddrBytes( - com.google.protobuf.ByteString value) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } @@ -2838,12 +2838,12 @@ public final class TestProtos { return this; } public final Builder setUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } public final Builder mergeUnknownFields( - final com.google.protobuf.UnknownFieldSet unknownFields) { + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { return super.mergeUnknownFields(unknownFields); } @@ -2861,22 +2861,22 @@ public final class TestProtos { return DEFAULT_INSTANCE; } - @java.lang.Deprecated public static final com.google.protobuf.Parser - PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { public AddrResponseProto parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { return new AddrResponseProto(input, extensionRegistry); } }; - public static com.google.protobuf.Parser parser() { + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { return PARSER; } @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -2886,42 +2886,42 @@ public final class TestProtos { } - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_EmptyRequestProto_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_EmptyRequestProto_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_EmptyResponseProto_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_EmptyResponseProto_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_EchoRequestProto_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_EchoRequestProto_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_EchoResponseProto_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_EchoResponseProto_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_PauseRequestProto_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_PauseRequestProto_fieldAccessorTable; - private static final com.google.protobuf.Descriptors.Descriptor + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_AddrResponseProto_descriptor; private static final - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_AddrResponseProto_fieldAccessorTable; - public static com.google.protobuf.Descriptors.FileDescriptor + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { return descriptor; } - private static com.google.protobuf.Descriptors.FileDescriptor + private static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor descriptor; static { java.lang.String[] descriptorData = { @@ -2933,52 +2933,52 @@ public final class TestProtos { "\n5org.apache.hadoop.hbase.shaded.ipc.pro" + "tobuf.generatedB\nTestProtos\240\001\001" }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; return null; } }; - com.google.protobuf.Descriptors.FileDescriptor + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { + new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] { }, assigner); internal_static_EmptyRequestProto_descriptor = getDescriptor().getMessageTypes().get(0); internal_static_EmptyRequestProto_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_EmptyRequestProto_descriptor, new java.lang.String[] { }); internal_static_EmptyResponseProto_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_EmptyResponseProto_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_EmptyResponseProto_descriptor, new java.lang.String[] { }); internal_static_EchoRequestProto_descriptor = getDescriptor().getMessageTypes().get(2); internal_static_EchoRequestProto_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_EchoRequestProto_descriptor, new java.lang.String[] { "Message", }); internal_static_EchoResponseProto_descriptor = getDescriptor().getMessageTypes().get(3); internal_static_EchoResponseProto_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_EchoResponseProto_descriptor, new java.lang.String[] { "Message", }); internal_static_PauseRequestProto_descriptor = getDescriptor().getMessageTypes().get(4); internal_static_PauseRequestProto_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_PauseRequestProto_descriptor, new java.lang.String[] { "Ms", }); internal_static_AddrResponseProto_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_AddrResponseProto_fieldAccessorTable = new - com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_AddrResponseProto_descriptor, new java.lang.String[] { "Addr", }); } diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestRpcServiceProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestRpcServiceProtos.java index 86d12f3..200a7f4 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestRpcServiceProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestRpcServiceProtos.java @@ -6,13 +6,13 @@ package org.apache.hadoop.hbase.shaded.ipc.protobuf.generated; public final class TestRpcServiceProtos { private TestRpcServiceProtos() {} public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistryLite registry) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) { } public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) { registerAllExtensions( - (com.google.protobuf.ExtensionRegistryLite) registry); + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry); } /** *
@@ -23,7 +23,7 @@ public final class TestRpcServiceProtos {
    * Protobuf service {@code TestProtobufRpcProto}
    */
   public static abstract class TestProtobufRpcProto
-      implements com.google.protobuf.Service {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.Service {
     protected TestProtobufRpcProto() {}
 
     public interface Interface {
@@ -31,103 +31,103 @@ public final class TestRpcServiceProtos {
        * rpc ping(.EmptyRequestProto) returns (.EmptyResponseProto);
        */
       public abstract void ping(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc echo(.EchoRequestProto) returns (.EchoResponseProto);
        */
       public abstract void echo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc error(.EmptyRequestProto) returns (.EmptyResponseProto);
        */
       public abstract void error(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc pause(.PauseRequestProto) returns (.EmptyResponseProto);
        */
       public abstract void pause(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc addr(.EmptyRequestProto) returns (.AddrResponseProto);
        */
       public abstract void addr(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     }
 
-    public static com.google.protobuf.Service newReflectiveService(
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
         final Interface impl) {
       return new TestProtobufRpcProto() {
         @java.lang.Override
         public  void ping(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.ping(controller, request, done);
         }
 
         @java.lang.Override
         public  void echo(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.echo(controller, request, done);
         }
 
         @java.lang.Override
         public  void error(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.error(controller, request, done);
         }
 
         @java.lang.Override
         public  void pause(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.pause(controller, request, done);
         }
 
         @java.lang.Override
         public  void addr(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.addr(controller, request, done);
         }
 
       };
     }
 
-    public static com.google.protobuf.BlockingService
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
         newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService() {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
             getDescriptorForType() {
           return getDescriptor();
         }
 
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message callBlockingMethod(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.callBlockingMethod() given method descriptor for " +
@@ -149,9 +149,9 @@ public final class TestRpcServiceProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getRequestPrototype() given method " +
@@ -173,9 +173,9 @@ public final class TestRpcServiceProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getResponsePrototype() given method " +
@@ -204,58 +204,58 @@ public final class TestRpcServiceProtos {
      * rpc ping(.EmptyRequestProto) returns (.EmptyResponseProto);
      */
     public abstract void ping(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc echo(.EchoRequestProto) returns (.EchoResponseProto);
      */
     public abstract void echo(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc error(.EmptyRequestProto) returns (.EmptyResponseProto);
      */
     public abstract void error(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc pause(.PauseRequestProto) returns (.EmptyResponseProto);
      */
     public abstract void pause(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc addr(.EmptyRequestProto) returns (.AddrResponseProto);
      */
     public abstract void addr(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.getDescriptor().getServices().get(0);
     }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
 
     public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Message> done) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.callMethod() given method descriptor for wrong " +
@@ -264,27 +264,27 @@ public final class TestRpcServiceProtos {
       switch(method.getIndex()) {
         case 0:
           this.ping(controller, (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 1:
           this.echo(controller, (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 2:
           this.error(controller, (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 3:
           this.pause(controller, (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 4:
           this.addr(controller, (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         default:
@@ -292,9 +292,9 @@ public final class TestRpcServiceProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getRequestPrototype() given method " +
@@ -316,9 +316,9 @@ public final class TestRpcServiceProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getResponsePrototype() given method " +
@@ -341,91 +341,91 @@ public final class TestRpcServiceProtos {
     }
 
     public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
       return new Stub(channel);
     }
 
     public static final class Stub extends org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
+      private Stub(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.RpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel;
 
-      public com.google.protobuf.RpcChannel getChannel() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel getChannel() {
         return channel;
       }
 
       public  void ping(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(0),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()));
       }
 
       public  void echo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto.class,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto.getDefaultInstance()));
       }
 
       public  void error(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(2),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()));
       }
 
       public  void pause(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(3),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.class,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto.getDefaultInstance()));
       }
 
       public  void addr(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(4),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto.class,
             org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto.getDefaultInstance()));
@@ -433,48 +433,48 @@ public final class TestRpcServiceProtos {
     }
 
     public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
       return new BlockingStub(channel);
     }
 
     public interface BlockingInterface {
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto ping(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto echo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto error(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto pause(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto addr(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+      private BlockingStub(org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.BlockingRpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel;
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto ping(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto) channel.callBlockingMethod(
           getDescriptor().getMethods().get(0),
           controller,
@@ -484,9 +484,9 @@ public final class TestRpcServiceProtos {
 
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto echo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto) channel.callBlockingMethod(
           getDescriptor().getMethods().get(1),
           controller,
@@ -496,9 +496,9 @@ public final class TestRpcServiceProtos {
 
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto error(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto) channel.callBlockingMethod(
           getDescriptor().getMethods().get(2),
           controller,
@@ -508,9 +508,9 @@ public final class TestRpcServiceProtos {
 
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto pause(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto) channel.callBlockingMethod(
           getDescriptor().getMethods().get(3),
           controller,
@@ -520,9 +520,9 @@ public final class TestRpcServiceProtos {
 
 
       public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto addr(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto) channel.callBlockingMethod(
           getDescriptor().getMethods().get(4),
           controller,
@@ -536,11 +536,11 @@ public final class TestRpcServiceProtos {
   }
 
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -555,17 +555,17 @@ public final class TestRpcServiceProtos {
       "hadoop.hbase.shaded.ipc.protobuf.generat" +
       "edB\024TestRpcServiceProtos\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.getDescriptor(),
         }, assigner);
     org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index e15dbc4..20020d4 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class AdminProtos {
   private AdminProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface GetRegionInfoRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetRegionInfoRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -44,11 +44,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.GetRegionInfoRequest}
    */
   public  static final class GetRegionInfoRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetRegionInfoRequest)
       GetRegionInfoRequestOrBuilder {
     // Use GetRegionInfoRequest.newBuilder() to construct.
-    private GetRegionInfoRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetRegionInfoRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetRegionInfoRequest() {
@@ -56,18 +56,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetRegionInfoRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -103,22 +103,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -180,7 +180,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -197,11 +197,11 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, compactionState_);
       }
       size += unknownFields.getSerializedSize();
@@ -248,7 +248,7 @@ public final class AdminProtos {
       }
       if (hasCompactionState()) {
         hash = (37 * hash) + COMPACTION_STATE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCompactionState());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -257,61 +257,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -329,7 +329,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -337,15 +337,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetRegionInfoRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetRegionInfoRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -358,12 +358,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -381,7 +381,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoRequest_descriptor;
       }
@@ -423,29 +423,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest)other);
         } else {
@@ -478,13 +478,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -497,7 +497,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -600,11 +600,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -646,12 +646,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -669,22 +669,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetRegionInfoRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetRegionInfoRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -696,7 +696,7 @@ public final class AdminProtos {
 
   public interface GetRegionInfoResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetRegionInfoResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionInfo region_info = 1;
@@ -733,11 +733,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.GetRegionInfoResponse}
    */
   public  static final class GetRegionInfoResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetRegionInfoResponse)
       GetRegionInfoResponseOrBuilder {
     // Use GetRegionInfoResponse.newBuilder() to construct.
-    private GetRegionInfoResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetRegionInfoResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetRegionInfoResponse() {
@@ -746,18 +746,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetRegionInfoResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -804,22 +804,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -830,7 +830,7 @@ public final class AdminProtos {
      * Protobuf enum {@code hbase.pb.GetRegionInfoResponse.CompactionState}
      */
     public enum CompactionState
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * NONE = 0;
        */
@@ -889,27 +889,27 @@ public final class AdminProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           CompactionState> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public CompactionState findValueByNumber(int number) {
                 return CompactionState.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.getDescriptor().getEnumTypes().get(0);
       }
@@ -917,7 +917,7 @@ public final class AdminProtos {
       private static final CompactionState[] VALUES = values();
 
       public static CompactionState valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -1005,7 +1005,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegionInfo());
@@ -1025,15 +1025,15 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegionInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, compactionState_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, isRecovering_);
       }
       size += unknownFields.getSerializedSize();
@@ -1088,7 +1088,7 @@ public final class AdminProtos {
       }
       if (hasIsRecovering()) {
         hash = (37 * hash) + ISRECOVERING_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIsRecovering());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -1097,61 +1097,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1169,7 +1169,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1177,15 +1177,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetRegionInfoResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetRegionInfoResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1198,12 +1198,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionInfoFieldBuilder();
         }
@@ -1223,7 +1223,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetRegionInfoResponse_descriptor;
       }
@@ -1269,29 +1269,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse)other);
         } else {
@@ -1327,13 +1327,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1346,7 +1346,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
       /**
        * required .hbase.pb.RegionInfo region_info = 1;
@@ -1449,11 +1449,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionInfo region_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   getRegionInfo(),
                   getParentForChildren(),
@@ -1531,12 +1531,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1554,22 +1554,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetRegionInfoResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetRegionInfoResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1581,7 +1581,7 @@ public final class AdminProtos {
 
   public interface GetStoreFileRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetStoreFileRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -1599,7 +1599,7 @@ public final class AdminProtos {
     /**
      * repeated bytes family = 2;
      */
-    java.util.List getFamilyList();
+    java.util.List getFamilyList();
     /**
      * repeated bytes family = 2;
      */
@@ -1607,7 +1607,7 @@ public final class AdminProtos {
     /**
      * repeated bytes family = 2;
      */
-    com.google.protobuf.ByteString getFamily(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily(int index);
   }
   /**
    * 
@@ -1619,11 +1619,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.GetStoreFileRequest}
    */
   public  static final class GetStoreFileRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetStoreFileRequest)
       GetStoreFileRequestOrBuilder {
     // Use GetStoreFileRequest.newBuilder() to construct.
-    private GetStoreFileRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetStoreFileRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetStoreFileRequest() {
@@ -1631,18 +1631,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetStoreFileRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1673,7 +1673,7 @@ public final class AdminProtos {
             }
             case 18: {
               if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                family_ = new java.util.ArrayList();
+                family_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000002;
               }
               family_.add(input.readBytes());
@@ -1681,10 +1681,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -1694,12 +1694,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1729,11 +1729,11 @@ public final class AdminProtos {
     }
 
     public static final int FAMILY_FIELD_NUMBER = 2;
-    private java.util.List family_;
+    private java.util.List family_;
     /**
      * repeated bytes family = 2;
      */
-    public java.util.List
+    public java.util.List
         getFamilyList() {
       return family_;
     }
@@ -1746,7 +1746,7 @@ public final class AdminProtos {
     /**
      * repeated bytes family = 2;
      */
-    public com.google.protobuf.ByteString getFamily(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily(int index) {
       return family_.get(index);
     }
 
@@ -1768,7 +1768,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -1785,13 +1785,13 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       {
         int dataSize = 0;
         for (int i = 0; i < family_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(family_.get(i));
         }
         size += dataSize;
@@ -1846,61 +1846,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1918,7 +1918,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1932,15 +1932,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetStoreFileRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetStoreFileRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1953,12 +1953,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -1976,7 +1976,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileRequest_descriptor;
       }
@@ -2019,29 +2019,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest)other);
         } else {
@@ -2081,13 +2081,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2100,7 +2100,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -2203,11 +2203,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -2217,17 +2217,17 @@ public final class AdminProtos {
         return regionBuilder_;
       }
 
-      private java.util.List family_ = java.util.Collections.emptyList();
+      private java.util.List family_ = java.util.Collections.emptyList();
       private void ensureFamilyIsMutable() {
         if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          family_ = new java.util.ArrayList(family_);
+          family_ = new java.util.ArrayList(family_);
           bitField0_ |= 0x00000002;
          }
       }
       /**
        * repeated bytes family = 2;
        */
-      public java.util.List
+      public java.util.List
           getFamilyList() {
         return java.util.Collections.unmodifiableList(family_);
       }
@@ -2240,14 +2240,14 @@ public final class AdminProtos {
       /**
        * repeated bytes family = 2;
        */
-      public com.google.protobuf.ByteString getFamily(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily(int index) {
         return family_.get(index);
       }
       /**
        * repeated bytes family = 2;
        */
       public Builder setFamily(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2259,7 +2259,7 @@ public final class AdminProtos {
       /**
        * repeated bytes family = 2;
        */
-      public Builder addFamily(com.google.protobuf.ByteString value) {
+      public Builder addFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2272,9 +2272,9 @@ public final class AdminProtos {
        * repeated bytes family = 2;
        */
       public Builder addAllFamily(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureFamilyIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, family_);
         onChanged();
         return this;
@@ -2289,12 +2289,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2312,22 +2312,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetStoreFileRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetStoreFileRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2339,7 +2339,7 @@ public final class AdminProtos {
 
   public interface GetStoreFileResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetStoreFileResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated string store_file = 1;
@@ -2357,37 +2357,37 @@ public final class AdminProtos {
     /**
      * repeated string store_file = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreFileBytes(int index);
   }
   /**
    * Protobuf type {@code hbase.pb.GetStoreFileResponse}
    */
   public  static final class GetStoreFileResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetStoreFileResponse)
       GetStoreFileResponseOrBuilder {
     // Use GetStoreFileResponse.newBuilder() to construct.
-    private GetStoreFileResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetStoreFileResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetStoreFileResponse() {
-      storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetStoreFileResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2404,9 +2404,9 @@ public final class AdminProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                storeFile_ = new com.google.protobuf.LazyStringArrayList();
+                storeFile_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                 mutable_bitField0_ |= 0x00000001;
               }
               storeFile_.add(bs);
@@ -2414,10 +2414,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -2427,12 +2427,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2440,11 +2440,11 @@ public final class AdminProtos {
     }
 
     public static final int STORE_FILE_FIELD_NUMBER = 1;
-    private com.google.protobuf.LazyStringList storeFile_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList storeFile_;
     /**
      * repeated string store_file = 1;
      */
-    public com.google.protobuf.ProtocolStringList
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
         getStoreFileList() {
       return storeFile_;
     }
@@ -2463,7 +2463,7 @@ public final class AdminProtos {
     /**
      * repeated string store_file = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreFileBytes(int index) {
       return storeFile_.getByteString(index);
     }
@@ -2478,10 +2478,10 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < storeFile_.size(); i++) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, storeFile_.getRaw(i));
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, storeFile_.getRaw(i));
       }
       unknownFields.writeTo(output);
     }
@@ -2539,61 +2539,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2611,7 +2611,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2619,15 +2619,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetStoreFileResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetStoreFileResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2640,23 +2640,23 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetStoreFileResponse_descriptor;
       }
@@ -2689,29 +2689,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse)other);
         } else {
@@ -2742,13 +2742,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2760,17 +2760,17 @@ public final class AdminProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.LazyStringList storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureStoreFileIsMutable() {
         if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          storeFile_ = new com.google.protobuf.LazyStringArrayList(storeFile_);
+          storeFile_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(storeFile_);
           bitField0_ |= 0x00000001;
          }
       }
       /**
        * repeated string store_file = 1;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getStoreFileList() {
         return storeFile_.getUnmodifiableView();
       }
@@ -2789,7 +2789,7 @@ public final class AdminProtos {
       /**
        * repeated string store_file = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStoreFileBytes(int index) {
         return storeFile_.getByteString(index);
       }
@@ -2825,7 +2825,7 @@ public final class AdminProtos {
       public Builder addAllStoreFile(
           java.lang.Iterable values) {
         ensureStoreFileIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, storeFile_);
         onChanged();
         return this;
@@ -2834,7 +2834,7 @@ public final class AdminProtos {
        * repeated string store_file = 1;
        */
       public Builder clearStoreFile() {
-        storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         onChanged();
         return this;
@@ -2843,7 +2843,7 @@ public final class AdminProtos {
        * repeated string store_file = 1;
        */
       public Builder addStoreFileBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2853,12 +2853,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2876,22 +2876,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetStoreFileResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetStoreFileResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2903,34 +2903,34 @@ public final class AdminProtos {
 
   public interface GetOnlineRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetOnlineRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.GetOnlineRegionRequest}
    */
   public  static final class GetOnlineRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetOnlineRegionRequest)
       GetOnlineRegionRequestOrBuilder {
     // Use GetOnlineRegionRequest.newBuilder() to construct.
-    private GetOnlineRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetOnlineRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetOnlineRegionRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetOnlineRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2948,22 +2948,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2980,7 +2980,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -3024,61 +3024,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3096,7 +3096,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3104,15 +3104,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetOnlineRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetOnlineRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3125,12 +3125,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3139,7 +3139,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionRequest_descriptor;
       }
@@ -3166,29 +3166,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest)other);
         } else {
@@ -3209,13 +3209,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3226,12 +3226,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3249,22 +3249,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetOnlineRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetOnlineRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3276,7 +3276,7 @@ public final class AdminProtos {
 
   public interface GetOnlineRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetOnlineRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.RegionInfo region_info = 1;
@@ -3306,11 +3306,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.GetOnlineRegionResponse}
    */
   public  static final class GetOnlineRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetOnlineRegionResponse)
       GetOnlineRegionResponseOrBuilder {
     // Use GetOnlineRegionResponse.newBuilder() to construct.
-    private GetOnlineRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetOnlineRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetOnlineRegionResponse() {
@@ -3318,18 +3318,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetOnlineRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3356,10 +3356,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -3369,12 +3369,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3432,7 +3432,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < regionInfo_.size(); i++) {
         output.writeMessage(1, regionInfo_.get(i));
@@ -3446,7 +3446,7 @@ public final class AdminProtos {
 
       size = 0;
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, regionInfo_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -3489,61 +3489,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3561,7 +3561,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3569,15 +3569,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetOnlineRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetOnlineRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3590,12 +3590,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionInfoFieldBuilder();
         }
@@ -3611,7 +3611,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetOnlineRegionResponse_descriptor;
       }
@@ -3648,29 +3648,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse)other);
         } else {
@@ -3700,7 +3700,7 @@ public final class AdminProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000001);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -3722,13 +3722,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3749,7 +3749,7 @@ public final class AdminProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -3881,7 +3881,7 @@ public final class AdminProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -3965,11 +3965,11 @@ public final class AdminProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -3980,12 +3980,12 @@ public final class AdminProtos {
         return regionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4003,22 +4003,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetOnlineRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetOnlineRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4030,7 +4030,7 @@ public final class AdminProtos {
 
   public interface OpenRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.OpenRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.OpenRegionRequest.RegionOpenInfo open_info = 1;
@@ -4094,11 +4094,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.OpenRegionRequest}
    */
   public  static final class OpenRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.OpenRegionRequest)
       OpenRegionRequestOrBuilder {
     // Use OpenRegionRequest.newBuilder() to construct.
-    private OpenRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private OpenRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private OpenRegionRequest() {
@@ -4108,18 +4108,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private OpenRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4156,10 +4156,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -4169,12 +4169,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4183,7 +4183,7 @@ public final class AdminProtos {
 
     public interface RegionOpenInfoOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.OpenRegionRequest.RegionOpenInfo)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required .hbase.pb.RegionInfo region = 1;
@@ -4252,11 +4252,11 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.OpenRegionRequest.RegionOpenInfo}
      */
     public  static final class RegionOpenInfo extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.OpenRegionRequest.RegionOpenInfo)
         RegionOpenInfoOrBuilder {
       // Use RegionOpenInfo.newBuilder() to construct.
-      private RegionOpenInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private RegionOpenInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private RegionOpenInfo() {
@@ -4266,18 +4266,18 @@ public final class AdminProtos {
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private RegionOpenInfo(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -4327,10 +4327,10 @@ public final class AdminProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -4340,12 +4340,12 @@ public final class AdminProtos {
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4471,7 +4471,7 @@ public final class AdminProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeMessage(1, getRegion());
@@ -4494,19 +4494,19 @@ public final class AdminProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(1, getRegion());
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt32Size(2, versionOfOfflineNode_);
         }
         for (int i = 0; i < favoredNodes_.size(); i++) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(3, favoredNodes_.get(i));
         }
         if (((bitField0_ & 0x00000004) == 0x00000004)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBoolSize(4, openForDistributedLogReplay_);
         }
         size += unknownFields.getSerializedSize();
@@ -4568,7 +4568,7 @@ public final class AdminProtos {
         }
         if (hasOpenForDistributedLogReplay()) {
           hash = (37 * hash) + OPENFORDISTRIBUTEDLOGREPLAY_FIELD_NUMBER;
-          hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+          hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
               getOpenForDistributedLogReplay());
         }
         hash = (29 * hash) + unknownFields.hashCode();
@@ -4577,61 +4577,61 @@ public final class AdminProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -4649,7 +4649,7 @@ public final class AdminProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -4657,15 +4657,15 @@ public final class AdminProtos {
        * Protobuf type {@code hbase.pb.OpenRegionRequest.RegionOpenInfo}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.OpenRegionRequest.RegionOpenInfo)
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfoOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -4678,12 +4678,12 @@ public final class AdminProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
             getRegionFieldBuilder();
             getFavoredNodesFieldBuilder();
@@ -4710,7 +4710,7 @@ public final class AdminProtos {
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_descriptor;
         }
@@ -4765,29 +4765,29 @@ public final class AdminProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo)other);
           } else {
@@ -4823,7 +4823,7 @@ public final class AdminProtos {
                 favoredNodes_ = other.favoredNodes_;
                 bitField0_ = (bitField0_ & ~0x00000004);
                 favoredNodesBuilder_ = 
-                  com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                  org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                      getFavoredNodesFieldBuilder() : null;
               } else {
                 favoredNodesBuilder_.addAllMessages(other.favoredNodes_);
@@ -4854,13 +4854,13 @@ public final class AdminProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -4873,7 +4873,7 @@ public final class AdminProtos {
         private int bitField0_;
 
         private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_ = null;
-        private com.google.protobuf.SingleFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionBuilder_;
         /**
          * required .hbase.pb.RegionInfo region = 1;
@@ -4976,11 +4976,11 @@ public final class AdminProtos {
         /**
          * required .hbase.pb.RegionInfo region = 1;
          */
-        private com.google.protobuf.SingleFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
             getRegionFieldBuilder() {
           if (regionBuilder_ == null) {
-            regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+            regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                     getRegion(),
                     getParentForChildren(),
@@ -5031,7 +5031,7 @@ public final class AdminProtos {
            }
         }
 
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> favoredNodesBuilder_;
 
         /**
@@ -5163,7 +5163,7 @@ public final class AdminProtos {
             java.lang.Iterable values) {
           if (favoredNodesBuilder_ == null) {
             ensureFavoredNodesIsMutable();
-            com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
                 values, favoredNodes_);
             onChanged();
           } else {
@@ -5247,11 +5247,11 @@ public final class AdminProtos {
              getFavoredNodesBuilderList() {
           return getFavoredNodesFieldBuilder().getBuilderList();
         }
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
             getFavoredNodesFieldBuilder() {
           if (favoredNodesBuilder_ == null) {
-            favoredNodesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+            favoredNodesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                     favoredNodes_,
                     ((bitField0_ & 0x00000004) == 0x00000004),
@@ -5310,12 +5310,12 @@ public final class AdminProtos {
           return this;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -5333,22 +5333,22 @@ public final class AdminProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public RegionOpenInfo parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new RegionOpenInfo(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -5456,7 +5456,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < openInfo_.size(); i++) {
         output.writeMessage(1, openInfo_.get(i));
@@ -5476,15 +5476,15 @@ public final class AdminProtos {
 
       size = 0;
       for (int i = 0; i < openInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, openInfo_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, serverStartCode_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(5, masterSystemTime_);
       }
       size += unknownFields.getSerializedSize();
@@ -5533,12 +5533,12 @@ public final class AdminProtos {
       }
       if (hasServerStartCode()) {
         hash = (37 * hash) + SERVERSTARTCODE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getServerStartCode());
       }
       if (hasMasterSystemTime()) {
         hash = (37 * hash) + MASTER_SYSTEM_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getMasterSystemTime());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -5547,61 +5547,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5619,7 +5619,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5627,15 +5627,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.OpenRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.OpenRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5648,12 +5648,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getOpenInfoFieldBuilder();
         }
@@ -5673,7 +5673,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionRequest_descriptor;
       }
@@ -5720,29 +5720,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest)other);
         } else {
@@ -5772,7 +5772,7 @@ public final class AdminProtos {
               openInfo_ = other.openInfo_;
               bitField0_ = (bitField0_ & ~0x00000001);
               openInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getOpenInfoFieldBuilder() : null;
             } else {
               openInfoBuilder_.addAllMessages(other.openInfo_);
@@ -5800,13 +5800,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5827,7 +5827,7 @@ public final class AdminProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfoOrBuilder> openInfoBuilder_;
 
       /**
@@ -5959,7 +5959,7 @@ public final class AdminProtos {
           java.lang.Iterable values) {
         if (openInfoBuilder_ == null) {
           ensureOpenInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, openInfo_);
           onChanged();
         } else {
@@ -6043,11 +6043,11 @@ public final class AdminProtos {
            getOpenInfoBuilderList() {
         return getOpenInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfoOrBuilder> 
           getOpenInfoFieldBuilder() {
         if (openInfoBuilder_ == null) {
-          openInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          openInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfoOrBuilder>(
                   openInfo_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -6154,12 +6154,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6177,22 +6177,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public OpenRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new OpenRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6204,7 +6204,7 @@ public final class AdminProtos {
 
   public interface OpenRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.OpenRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.OpenRegionResponse.RegionOpeningState opening_state = 1;
@@ -6223,11 +6223,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.OpenRegionResponse}
    */
   public  static final class OpenRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.OpenRegionResponse)
       OpenRegionResponseOrBuilder {
     // Use OpenRegionResponse.newBuilder() to construct.
-    private OpenRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private OpenRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private OpenRegionResponse() {
@@ -6235,18 +6235,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private OpenRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6297,10 +6297,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -6310,12 +6310,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6326,7 +6326,7 @@ public final class AdminProtos {
      * Protobuf enum {@code hbase.pb.OpenRegionResponse.RegionOpeningState}
      */
     public enum RegionOpeningState
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * OPENED = 0;
        */
@@ -6376,27 +6376,27 @@ public final class AdminProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           RegionOpeningState> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public RegionOpeningState findValueByNumber(int number) {
                 return RegionOpeningState.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.getDescriptor().getEnumTypes().get(0);
       }
@@ -6404,7 +6404,7 @@ public final class AdminProtos {
       private static final RegionOpeningState[] VALUES = values();
 
       public static RegionOpeningState valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -6423,9 +6423,9 @@ public final class AdminProtos {
 
     public static final int OPENING_STATE_FIELD_NUMBER = 1;
     private java.util.List openingState_;
-    private static final com.google.protobuf.Internal.ListAdapter.Converter<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter.Converter<
         java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState> openingState_converter_ =
-            new com.google.protobuf.Internal.ListAdapter.Converter<
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter.Converter<
                 java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState>() {
               public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState convert(java.lang.Integer from) {
                 org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState result = org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState.valueOf(from);
@@ -6436,7 +6436,7 @@ public final class AdminProtos {
      * repeated .hbase.pb.OpenRegionResponse.RegionOpeningState opening_state = 1;
      */
     public java.util.List getOpeningStateList() {
-      return new com.google.protobuf.Internal.ListAdapter<
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter<
           java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState>(openingState_, openingState_converter_);
     }
     /**
@@ -6462,7 +6462,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < openingState_.size(); i++) {
         output.writeEnum(1, openingState_.get(i));
@@ -6478,7 +6478,7 @@ public final class AdminProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < openingState_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeEnumSizeNoTag(openingState_.get(i));
         }
         size += dataSize;
@@ -6523,61 +6523,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6595,7 +6595,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6603,15 +6603,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.OpenRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.OpenRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6624,12 +6624,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -6640,7 +6640,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_OpenRegionResponse_descriptor;
       }
@@ -6673,29 +6673,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse)other);
         } else {
@@ -6726,13 +6726,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6756,7 +6756,7 @@ public final class AdminProtos {
        * repeated .hbase.pb.OpenRegionResponse.RegionOpeningState opening_state = 1;
        */
       public java.util.List getOpeningStateList() {
-        return new com.google.protobuf.Internal.ListAdapter<
+        return new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter<
             java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState>(openingState_, openingState_converter_);
       }
       /**
@@ -6818,12 +6818,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6841,22 +6841,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public OpenRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new OpenRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6868,7 +6868,7 @@ public final class AdminProtos {
 
   public interface WarmupRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WarmupRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionInfo regionInfo = 1;
@@ -6887,29 +6887,29 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.WarmupRegionRequest}
    */
   public  static final class WarmupRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WarmupRegionRequest)
       WarmupRegionRequestOrBuilder {
     // Use WarmupRegionRequest.newBuilder() to construct.
-    private WarmupRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WarmupRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WarmupRegionRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WarmupRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6940,22 +6940,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7002,7 +7002,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegionInfo());
@@ -7016,7 +7016,7 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegionInfo());
       }
       size += unknownFields.getSerializedSize();
@@ -7062,61 +7062,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7134,7 +7134,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7142,15 +7142,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.WarmupRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WarmupRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7163,12 +7163,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionInfoFieldBuilder();
         }
@@ -7184,7 +7184,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionRequest_descriptor;
       }
@@ -7222,29 +7222,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest)other);
         } else {
@@ -7274,13 +7274,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7293,7 +7293,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
       /**
        * required .hbase.pb.RegionInfo regionInfo = 1;
@@ -7396,11 +7396,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionInfo regionInfo = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   getRegionInfo(),
                   getParentForChildren(),
@@ -7410,12 +7410,12 @@ public final class AdminProtos {
         return regionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7433,22 +7433,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WarmupRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WarmupRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7460,34 +7460,34 @@ public final class AdminProtos {
 
   public interface WarmupRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WarmupRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.WarmupRegionResponse}
    */
   public  static final class WarmupRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WarmupRegionResponse)
       WarmupRegionResponseOrBuilder {
     // Use WarmupRegionResponse.newBuilder() to construct.
-    private WarmupRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WarmupRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WarmupRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WarmupRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7505,22 +7505,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7537,7 +7537,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -7581,61 +7581,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7653,7 +7653,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7661,15 +7661,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.WarmupRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WarmupRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7682,12 +7682,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7696,7 +7696,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WarmupRegionResponse_descriptor;
       }
@@ -7723,29 +7723,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse)other);
         } else {
@@ -7766,13 +7766,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7783,12 +7783,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7806,22 +7806,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WarmupRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WarmupRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7833,7 +7833,7 @@ public final class AdminProtos {
 
   public interface CloseRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -7906,11 +7906,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.CloseRegionRequest}
    */
   public  static final class CloseRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionRequest)
       CloseRegionRequestOrBuilder {
     // Use CloseRegionRequest.newBuilder() to construct.
-    private CloseRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CloseRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CloseRegionRequest() {
@@ -7920,18 +7920,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CloseRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7990,22 +7990,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8132,7 +8132,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -8158,23 +8158,23 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, versionOfClosingNode_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, transitionInZK_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getDestinationServer());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(5, serverStartCode_);
       }
       size += unknownFields.getSerializedSize();
@@ -8240,7 +8240,7 @@ public final class AdminProtos {
       }
       if (hasTransitionInZK()) {
         hash = (37 * hash) + TRANSITION_IN_ZK_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getTransitionInZK());
       }
       if (hasDestinationServer()) {
@@ -8249,7 +8249,7 @@ public final class AdminProtos {
       }
       if (hasServerStartCode()) {
         hash = (37 * hash) + SERVERSTARTCODE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getServerStartCode());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -8258,61 +8258,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8330,7 +8330,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8344,15 +8344,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.CloseRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8365,12 +8365,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getDestinationServerFieldBuilder();
@@ -8399,7 +8399,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionRequest_descriptor;
       }
@@ -8457,29 +8457,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest)other);
         } else {
@@ -8526,13 +8526,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8545,7 +8545,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -8648,11 +8648,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -8727,7 +8727,7 @@ public final class AdminProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName destinationServer_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> destinationServerBuilder_;
       /**
        * optional .hbase.pb.ServerName destination_server = 4;
@@ -8830,11 +8830,11 @@ public final class AdminProtos {
       /**
        * optional .hbase.pb.ServerName destination_server = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getDestinationServerFieldBuilder() {
         if (destinationServerBuilder_ == null) {
-          destinationServerBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          destinationServerBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getDestinationServer(),
                   getParentForChildren(),
@@ -8892,12 +8892,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8915,22 +8915,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CloseRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CloseRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8942,7 +8942,7 @@ public final class AdminProtos {
 
   public interface CloseRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool closed = 1;
@@ -8957,11 +8957,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.CloseRegionResponse}
    */
   public  static final class CloseRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionResponse)
       CloseRegionResponseOrBuilder {
     // Use CloseRegionResponse.newBuilder() to construct.
-    private CloseRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CloseRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CloseRegionResponse() {
@@ -8969,18 +8969,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CloseRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9003,22 +9003,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9055,7 +9055,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, closed_);
@@ -9069,7 +9069,7 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, closed_);
       }
       size += unknownFields.getSerializedSize();
@@ -9107,7 +9107,7 @@ public final class AdminProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasClosed()) {
         hash = (37 * hash) + CLOSED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getClosed());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -9116,61 +9116,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9188,7 +9188,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9196,15 +9196,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.CloseRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9217,12 +9217,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -9233,7 +9233,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionResponse_descriptor;
       }
@@ -9267,29 +9267,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse)other);
         } else {
@@ -9316,13 +9316,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9366,12 +9366,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9389,22 +9389,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CloseRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CloseRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9416,7 +9416,7 @@ public final class AdminProtos {
 
   public interface FlushRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -9468,11 +9468,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.FlushRegionRequest}
    */
   public  static final class FlushRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FlushRegionRequest)
       FlushRegionRequestOrBuilder {
     // Use FlushRegionRequest.newBuilder() to construct.
-    private FlushRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FlushRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FlushRegionRequest() {
@@ -9481,18 +9481,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FlushRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9533,22 +9533,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9633,7 +9633,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -9653,15 +9653,15 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, ifOlderThanTs_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, writeFlushWalMarker_);
       }
       size += unknownFields.getSerializedSize();
@@ -9713,12 +9713,12 @@ public final class AdminProtos {
       }
       if (hasIfOlderThanTs()) {
         hash = (37 * hash) + IF_OLDER_THAN_TS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getIfOlderThanTs());
       }
       if (hasWriteFlushWalMarker()) {
         hash = (37 * hash) + WRITE_FLUSH_WAL_MARKER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getWriteFlushWalMarker());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -9727,61 +9727,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9799,7 +9799,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9814,15 +9814,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.FlushRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FlushRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9835,12 +9835,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -9860,7 +9860,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
       }
@@ -9906,29 +9906,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)other);
         } else {
@@ -9964,13 +9964,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9983,7 +9983,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -10086,11 +10086,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -10180,12 +10180,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10203,22 +10203,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FlushRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FlushRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10230,7 +10230,7 @@ public final class AdminProtos {
 
   public interface FlushRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 last_flush_time = 1;
@@ -10263,11 +10263,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.FlushRegionResponse}
    */
   public  static final class FlushRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FlushRegionResponse)
       FlushRegionResponseOrBuilder {
     // Use FlushRegionResponse.newBuilder() to construct.
-    private FlushRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FlushRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FlushRegionResponse() {
@@ -10277,18 +10277,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FlushRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10321,22 +10321,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10403,7 +10403,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, lastFlushTime_);
@@ -10423,15 +10423,15 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, lastFlushTime_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, flushed_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, wroteFlushWalMarker_);
       }
       size += unknownFields.getSerializedSize();
@@ -10479,17 +10479,17 @@ public final class AdminProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLastFlushTime()) {
         hash = (37 * hash) + LAST_FLUSH_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastFlushTime());
       }
       if (hasFlushed()) {
         hash = (37 * hash) + FLUSHED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getFlushed());
       }
       if (hasWroteFlushWalMarker()) {
         hash = (37 * hash) + WROTE_FLUSH_WAL_MARKER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getWroteFlushWalMarker());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -10498,61 +10498,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10570,7 +10570,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10578,15 +10578,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.FlushRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FlushRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10599,12 +10599,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -10619,7 +10619,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_descriptor;
       }
@@ -10661,29 +10661,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse)other);
         } else {
@@ -10716,13 +10716,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -10830,12 +10830,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10853,22 +10853,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FlushRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FlushRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10880,7 +10880,7 @@ public final class AdminProtos {
 
   public interface SplitRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SplitRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -10902,7 +10902,7 @@ public final class AdminProtos {
     /**
      * optional bytes split_point = 2;
      */
-    com.google.protobuf.ByteString getSplitPoint();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitPoint();
   }
   /**
    * 
@@ -10917,30 +10917,30 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.SplitRegionRequest}
    */
   public  static final class SplitRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SplitRegionRequest)
       SplitRegionRequestOrBuilder {
     // Use SplitRegionRequest.newBuilder() to construct.
-    private SplitRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SplitRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SplitRegionRequest() {
-      splitPoint_ = com.google.protobuf.ByteString.EMPTY;
+      splitPoint_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SplitRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10976,22 +10976,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11021,7 +11021,7 @@ public final class AdminProtos {
     }
 
     public static final int SPLIT_POINT_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString splitPoint_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString splitPoint_;
     /**
      * optional bytes split_point = 2;
      */
@@ -11031,7 +11031,7 @@ public final class AdminProtos {
     /**
      * optional bytes split_point = 2;
      */
-    public com.google.protobuf.ByteString getSplitPoint() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitPoint() {
       return splitPoint_;
     }
 
@@ -11053,7 +11053,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -11070,11 +11070,11 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, splitPoint_);
       }
       size += unknownFields.getSerializedSize();
@@ -11129,61 +11129,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11201,7 +11201,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11218,15 +11218,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.SplitRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SplitRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11239,12 +11239,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -11257,12 +11257,12 @@ public final class AdminProtos {
           regionBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        splitPoint_ = com.google.protobuf.ByteString.EMPTY;
+        splitPoint_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionRequest_descriptor;
       }
@@ -11304,29 +11304,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest)other);
         } else {
@@ -11359,13 +11359,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11378,7 +11378,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -11481,11 +11481,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -11495,7 +11495,7 @@ public final class AdminProtos {
         return regionBuilder_;
       }
 
-      private com.google.protobuf.ByteString splitPoint_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString splitPoint_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes split_point = 2;
        */
@@ -11505,13 +11505,13 @@ public final class AdminProtos {
       /**
        * optional bytes split_point = 2;
        */
-      public com.google.protobuf.ByteString getSplitPoint() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitPoint() {
         return splitPoint_;
       }
       /**
        * optional bytes split_point = 2;
        */
-      public Builder setSplitPoint(com.google.protobuf.ByteString value) {
+      public Builder setSplitPoint(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11530,12 +11530,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11553,22 +11553,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SplitRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SplitRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11580,34 +11580,34 @@ public final class AdminProtos {
 
   public interface SplitRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SplitRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.SplitRegionResponse}
    */
   public  static final class SplitRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SplitRegionResponse)
       SplitRegionResponseOrBuilder {
     // Use SplitRegionResponse.newBuilder() to construct.
-    private SplitRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SplitRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SplitRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SplitRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11625,22 +11625,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11657,7 +11657,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -11701,61 +11701,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11773,7 +11773,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11781,15 +11781,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.SplitRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SplitRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11802,12 +11802,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -11816,7 +11816,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionResponse_descriptor;
       }
@@ -11843,29 +11843,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse)other);
         } else {
@@ -11886,13 +11886,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11903,12 +11903,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11926,22 +11926,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SplitRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SplitRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11953,7 +11953,7 @@ public final class AdminProtos {
 
   public interface CompactRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CompactRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -11984,7 +11984,7 @@ public final class AdminProtos {
     /**
      * optional bytes family = 3;
      */
-    com.google.protobuf.ByteString getFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
   }
   /**
    * 
@@ -11997,31 +11997,31 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.CompactRegionRequest}
    */
   public  static final class CompactRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CompactRegionRequest)
       CompactRegionRequestOrBuilder {
     // Use CompactRegionRequest.newBuilder() to construct.
-    private CompactRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CompactRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CompactRegionRequest() {
       major_ = false;
-      family_ = com.google.protobuf.ByteString.EMPTY;
+      family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CompactRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12062,22 +12062,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12122,7 +12122,7 @@ public final class AdminProtos {
     }
 
     public static final int FAMILY_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString family_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
     /**
      * optional bytes family = 3;
      */
@@ -12132,7 +12132,7 @@ public final class AdminProtos {
     /**
      * optional bytes family = 3;
      */
-    public com.google.protobuf.ByteString getFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
@@ -12154,7 +12154,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -12174,15 +12174,15 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, major_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, family_);
       }
       size += unknownFields.getSerializedSize();
@@ -12234,7 +12234,7 @@ public final class AdminProtos {
       }
       if (hasMajor()) {
         hash = (37 * hash) + MAJOR_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getMajor());
       }
       if (hasFamily()) {
@@ -12247,61 +12247,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12319,7 +12319,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12334,15 +12334,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.CompactRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CompactRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12355,12 +12355,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -12375,12 +12375,12 @@ public final class AdminProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         major_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionRequest_descriptor;
       }
@@ -12426,29 +12426,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest)other);
         } else {
@@ -12484,13 +12484,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12503,7 +12503,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -12606,11 +12606,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -12652,7 +12652,7 @@ public final class AdminProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes family = 3;
        */
@@ -12662,13 +12662,13 @@ public final class AdminProtos {
       /**
        * optional bytes family = 3;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
        * optional bytes family = 3;
        */
-      public Builder setFamily(com.google.protobuf.ByteString value) {
+      public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -12687,12 +12687,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12710,22 +12710,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CompactRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CompactRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12737,34 +12737,34 @@ public final class AdminProtos {
 
   public interface CompactRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CompactRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.CompactRegionResponse}
    */
   public  static final class CompactRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CompactRegionResponse)
       CompactRegionResponseOrBuilder {
     // Use CompactRegionResponse.newBuilder() to construct.
-    private CompactRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CompactRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CompactRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CompactRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12782,22 +12782,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12814,7 +12814,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -12858,61 +12858,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12930,7 +12930,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12938,15 +12938,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.CompactRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CompactRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12959,12 +12959,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -12973,7 +12973,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CompactRegionResponse_descriptor;
       }
@@ -13000,29 +13000,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse)other);
         } else {
@@ -13043,13 +13043,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13060,12 +13060,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -13083,22 +13083,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CompactRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CompactRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13110,7 +13110,7 @@ public final class AdminProtos {
 
   public interface UpdateFavoredNodesRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UpdateFavoredNodesRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.UpdateFavoredNodesRequest.RegionUpdateInfo update_info = 1;
@@ -13140,11 +13140,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.UpdateFavoredNodesRequest}
    */
   public  static final class UpdateFavoredNodesRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UpdateFavoredNodesRequest)
       UpdateFavoredNodesRequestOrBuilder {
     // Use UpdateFavoredNodesRequest.newBuilder() to construct.
-    private UpdateFavoredNodesRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UpdateFavoredNodesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UpdateFavoredNodesRequest() {
@@ -13152,18 +13152,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UpdateFavoredNodesRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -13190,10 +13190,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -13203,12 +13203,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -13217,7 +13217,7 @@ public final class AdminProtos {
 
     public interface RegionUpdateInfoOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.UpdateFavoredNodesRequest.RegionUpdateInfo)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required .hbase.pb.RegionInfo region = 1;
@@ -13260,11 +13260,11 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.UpdateFavoredNodesRequest.RegionUpdateInfo}
      */
     public  static final class RegionUpdateInfo extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.UpdateFavoredNodesRequest.RegionUpdateInfo)
         RegionUpdateInfoOrBuilder {
       // Use RegionUpdateInfo.newBuilder() to construct.
-      private RegionUpdateInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private RegionUpdateInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private RegionUpdateInfo() {
@@ -13272,18 +13272,18 @@ public final class AdminProtos {
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private RegionUpdateInfo(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -13323,10 +13323,10 @@ public final class AdminProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -13336,12 +13336,12 @@ public final class AdminProtos {
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13429,7 +13429,7 @@ public final class AdminProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeMessage(1, getRegion());
@@ -13446,11 +13446,11 @@ public final class AdminProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(1, getRegion());
         }
         for (int i = 0; i < favoredNodes_.size(); i++) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(2, favoredNodes_.get(i));
         }
         size += unknownFields.getSerializedSize();
@@ -13502,61 +13502,61 @@ public final class AdminProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -13574,7 +13574,7 @@ public final class AdminProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -13582,15 +13582,15 @@ public final class AdminProtos {
        * Protobuf type {@code hbase.pb.UpdateFavoredNodesRequest.RegionUpdateInfo}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.UpdateFavoredNodesRequest.RegionUpdateInfo)
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfoOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -13603,12 +13603,12 @@ public final class AdminProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
             getRegionFieldBuilder();
             getFavoredNodesFieldBuilder();
@@ -13631,7 +13631,7 @@ public final class AdminProtos {
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor;
         }
@@ -13678,29 +13678,29 @@ public final class AdminProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo)other);
           } else {
@@ -13733,7 +13733,7 @@ public final class AdminProtos {
                 favoredNodes_ = other.favoredNodes_;
                 bitField0_ = (bitField0_ & ~0x00000002);
                 favoredNodesBuilder_ = 
-                  com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                  org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                      getFavoredNodesFieldBuilder() : null;
               } else {
                 favoredNodesBuilder_.addAllMessages(other.favoredNodes_);
@@ -13761,13 +13761,13 @@ public final class AdminProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -13780,7 +13780,7 @@ public final class AdminProtos {
         private int bitField0_;
 
         private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_ = null;
-        private com.google.protobuf.SingleFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionBuilder_;
         /**
          * required .hbase.pb.RegionInfo region = 1;
@@ -13883,11 +13883,11 @@ public final class AdminProtos {
         /**
          * required .hbase.pb.RegionInfo region = 1;
          */
-        private com.google.protobuf.SingleFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
             getRegionFieldBuilder() {
           if (regionBuilder_ == null) {
-            regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+            regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                     getRegion(),
                     getParentForChildren(),
@@ -13906,7 +13906,7 @@ public final class AdminProtos {
            }
         }
 
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> favoredNodesBuilder_;
 
         /**
@@ -14038,7 +14038,7 @@ public final class AdminProtos {
             java.lang.Iterable values) {
           if (favoredNodesBuilder_ == null) {
             ensureFavoredNodesIsMutable();
-            com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
                 values, favoredNodes_);
             onChanged();
           } else {
@@ -14122,11 +14122,11 @@ public final class AdminProtos {
              getFavoredNodesBuilderList() {
           return getFavoredNodesFieldBuilder().getBuilderList();
         }
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
             getFavoredNodesFieldBuilder() {
           if (favoredNodesBuilder_ == null) {
-            favoredNodesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+            favoredNodesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                     favoredNodes_,
                     ((bitField0_ & 0x00000002) == 0x00000002),
@@ -14137,12 +14137,12 @@ public final class AdminProtos {
           return favoredNodesBuilder_;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -14160,22 +14160,22 @@ public final class AdminProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public RegionUpdateInfo parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new RegionUpdateInfo(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -14236,7 +14236,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < updateInfo_.size(); i++) {
         output.writeMessage(1, updateInfo_.get(i));
@@ -14250,7 +14250,7 @@ public final class AdminProtos {
 
       size = 0;
       for (int i = 0; i < updateInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, updateInfo_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -14293,61 +14293,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14365,7 +14365,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -14373,15 +14373,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.UpdateFavoredNodesRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UpdateFavoredNodesRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -14394,12 +14394,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUpdateInfoFieldBuilder();
         }
@@ -14415,7 +14415,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor;
       }
@@ -14452,29 +14452,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)other);
         } else {
@@ -14504,7 +14504,7 @@ public final class AdminProtos {
               updateInfo_ = other.updateInfo_;
               bitField0_ = (bitField0_ & ~0x00000001);
               updateInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getUpdateInfoFieldBuilder() : null;
             } else {
               updateInfoBuilder_.addAllMessages(other.updateInfo_);
@@ -14526,13 +14526,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -14553,7 +14553,7 @@ public final class AdminProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfoOrBuilder> updateInfoBuilder_;
 
       /**
@@ -14685,7 +14685,7 @@ public final class AdminProtos {
           java.lang.Iterable values) {
         if (updateInfoBuilder_ == null) {
           ensureUpdateInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, updateInfo_);
           onChanged();
         } else {
@@ -14769,11 +14769,11 @@ public final class AdminProtos {
            getUpdateInfoBuilderList() {
         return getUpdateInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfoOrBuilder> 
           getUpdateInfoFieldBuilder() {
         if (updateInfoBuilder_ == null) {
-          updateInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          updateInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfoOrBuilder>(
                   updateInfo_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -14784,12 +14784,12 @@ public final class AdminProtos {
         return updateInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14807,22 +14807,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UpdateFavoredNodesRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UpdateFavoredNodesRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14834,7 +14834,7 @@ public final class AdminProtos {
 
   public interface UpdateFavoredNodesResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UpdateFavoredNodesResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint32 response = 1;
@@ -14849,11 +14849,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.UpdateFavoredNodesResponse}
    */
   public  static final class UpdateFavoredNodesResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UpdateFavoredNodesResponse)
       UpdateFavoredNodesResponseOrBuilder {
     // Use UpdateFavoredNodesResponse.newBuilder() to construct.
-    private UpdateFavoredNodesResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UpdateFavoredNodesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UpdateFavoredNodesResponse() {
@@ -14861,18 +14861,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UpdateFavoredNodesResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14895,22 +14895,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14943,7 +14943,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, response_);
@@ -14957,7 +14957,7 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, response_);
       }
       size += unknownFields.getSerializedSize();
@@ -15003,61 +15003,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15075,7 +15075,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15083,15 +15083,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.UpdateFavoredNodesResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UpdateFavoredNodesResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15104,12 +15104,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -15120,7 +15120,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateFavoredNodesResponse_descriptor;
       }
@@ -15154,29 +15154,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse)other);
         } else {
@@ -15200,13 +15200,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15250,12 +15250,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15273,22 +15273,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UpdateFavoredNodesResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UpdateFavoredNodesResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15300,7 +15300,7 @@ public final class AdminProtos {
 
   public interface MergeRegionsRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MergeRegionsRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region_a = 1;
@@ -15365,11 +15365,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.MergeRegionsRequest}
    */
   public  static final class MergeRegionsRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MergeRegionsRequest)
       MergeRegionsRequestOrBuilder {
     // Use MergeRegionsRequest.newBuilder() to construct.
-    private MergeRegionsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MergeRegionsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MergeRegionsRequest() {
@@ -15378,18 +15378,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MergeRegionsRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15443,22 +15443,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -15572,7 +15572,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegionA());
@@ -15595,19 +15595,19 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegionA());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getRegionB());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, forcible_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, masterSystemTime_);
       }
       size += unknownFields.getSerializedSize();
@@ -15668,12 +15668,12 @@ public final class AdminProtos {
       }
       if (hasForcible()) {
         hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getForcible());
       }
       if (hasMasterSystemTime()) {
         hash = (37 * hash) + MASTER_SYSTEM_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getMasterSystemTime());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -15682,61 +15682,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15754,7 +15754,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15769,15 +15769,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.MergeRegionsRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MergeRegionsRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15790,12 +15790,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionAFieldBuilder();
           getRegionBFieldBuilder();
@@ -15822,7 +15822,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsRequest_descriptor;
       }
@@ -15876,29 +15876,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest)other);
         } else {
@@ -15943,13 +15943,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15962,7 +15962,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionA_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionABuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region_a = 1;
@@ -16065,11 +16065,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region_a = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionAFieldBuilder() {
         if (regionABuilder_ == null) {
-          regionABuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionABuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegionA(),
                   getParentForChildren(),
@@ -16080,7 +16080,7 @@ public final class AdminProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionB_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region_b = 2;
@@ -16183,11 +16183,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.RegionSpecifier region_b = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionBFieldBuilder() {
         if (regionBBuilder_ == null) {
-          regionBBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegionB(),
                   getParentForChildren(),
@@ -16277,12 +16277,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16300,22 +16300,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MergeRegionsRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MergeRegionsRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16327,34 +16327,34 @@ public final class AdminProtos {
 
   public interface MergeRegionsResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MergeRegionsResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.MergeRegionsResponse}
    */
   public  static final class MergeRegionsResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MergeRegionsResponse)
       MergeRegionsResponseOrBuilder {
     // Use MergeRegionsResponse.newBuilder() to construct.
-    private MergeRegionsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MergeRegionsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MergeRegionsResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MergeRegionsResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16372,22 +16372,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16404,7 +16404,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -16448,61 +16448,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16520,7 +16520,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16528,15 +16528,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.MergeRegionsResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MergeRegionsResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16549,12 +16549,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -16563,7 +16563,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_MergeRegionsResponse_descriptor;
       }
@@ -16590,29 +16590,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse)other);
         } else {
@@ -16633,13 +16633,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16650,12 +16650,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16673,22 +16673,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MergeRegionsResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MergeRegionsResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16700,7 +16700,7 @@ public final class AdminProtos {
 
   public interface WALEntryOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WALEntry)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.WALKey key = 1;
@@ -16724,7 +16724,7 @@ public final class AdminProtos {
      *
      * repeated bytes key_value_bytes = 2;
      */
-    java.util.List getKeyValueBytesList();
+    java.util.List getKeyValueBytesList();
     /**
      * 
      * Following may be null if the KVs/Cells are carried along the side in a cellblock (See
@@ -16744,7 +16744,7 @@ public final class AdminProtos {
      *
      * repeated bytes key_value_bytes = 2;
      */
-    com.google.protobuf.ByteString getKeyValueBytes(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getKeyValueBytes(int index);
 
     /**
      * 
@@ -16771,11 +16771,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.WALEntry}
    */
   public  static final class WALEntry extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WALEntry)
       WALEntryOrBuilder {
     // Use WALEntry.newBuilder() to construct.
-    private WALEntry(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WALEntry(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WALEntry() {
@@ -16784,18 +16784,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WALEntry(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16826,7 +16826,7 @@ public final class AdminProtos {
             }
             case 18: {
               if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                keyValueBytes_ = new java.util.ArrayList();
+                keyValueBytes_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000002;
               }
               keyValueBytes_.add(input.readBytes());
@@ -16839,10 +16839,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -16852,12 +16852,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WALEntry_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WALEntry_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16887,7 +16887,7 @@ public final class AdminProtos {
     }
 
     public static final int KEY_VALUE_BYTES_FIELD_NUMBER = 2;
-    private java.util.List keyValueBytes_;
+    private java.util.List keyValueBytes_;
     /**
      * 
      * Following may be null if the KVs/Cells are carried along the side in a cellblock (See
@@ -16897,7 +16897,7 @@ public final class AdminProtos {
      *
      * repeated bytes key_value_bytes = 2;
      */
-    public java.util.List
+    public java.util.List
         getKeyValueBytesList() {
       return keyValueBytes_;
     }
@@ -16922,7 +16922,7 @@ public final class AdminProtos {
      *
      * repeated bytes key_value_bytes = 2;
      */
-    public com.google.protobuf.ByteString getKeyValueBytes(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getKeyValueBytes(int index) {
       return keyValueBytes_.get(index);
     }
 
@@ -16967,7 +16967,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getKey());
@@ -16987,20 +16987,20 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getKey());
       }
       {
         int dataSize = 0;
         for (int i = 0; i < keyValueBytes_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(keyValueBytes_.get(i));
         }
         size += dataSize;
         size += 1 * getKeyValueBytesList().size();
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(3, associatedCellCount_);
       }
       size += unknownFields.getSerializedSize();
@@ -17061,61 +17061,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -17133,7 +17133,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -17145,15 +17145,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.WALEntry}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WALEntry)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntryOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WALEntry_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WALEntry_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -17166,12 +17166,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getKeyFieldBuilder();
         }
@@ -17191,7 +17191,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_WALEntry_descriptor;
       }
@@ -17238,29 +17238,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry)other);
         } else {
@@ -17303,13 +17303,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -17322,7 +17322,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey key_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKeyOrBuilder> keyBuilder_;
       /**
        * required .hbase.pb.WALKey key = 1;
@@ -17425,11 +17425,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.WALKey key = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKeyOrBuilder> 
           getKeyFieldBuilder() {
         if (keyBuilder_ == null) {
-          keyBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          keyBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKeyOrBuilder>(
                   getKey(),
                   getParentForChildren(),
@@ -17439,10 +17439,10 @@ public final class AdminProtos {
         return keyBuilder_;
       }
 
-      private java.util.List keyValueBytes_ = java.util.Collections.emptyList();
+      private java.util.List keyValueBytes_ = java.util.Collections.emptyList();
       private void ensureKeyValueBytesIsMutable() {
         if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          keyValueBytes_ = new java.util.ArrayList(keyValueBytes_);
+          keyValueBytes_ = new java.util.ArrayList(keyValueBytes_);
           bitField0_ |= 0x00000002;
          }
       }
@@ -17455,7 +17455,7 @@ public final class AdminProtos {
        *
        * repeated bytes key_value_bytes = 2;
        */
-      public java.util.List
+      public java.util.List
           getKeyValueBytesList() {
         return java.util.Collections.unmodifiableList(keyValueBytes_);
       }
@@ -17480,7 +17480,7 @@ public final class AdminProtos {
        *
        * repeated bytes key_value_bytes = 2;
        */
-      public com.google.protobuf.ByteString getKeyValueBytes(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getKeyValueBytes(int index) {
         return keyValueBytes_.get(index);
       }
       /**
@@ -17493,7 +17493,7 @@ public final class AdminProtos {
        * repeated bytes key_value_bytes = 2;
        */
       public Builder setKeyValueBytes(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -17511,7 +17511,7 @@ public final class AdminProtos {
        *
        * repeated bytes key_value_bytes = 2;
        */
-      public Builder addKeyValueBytes(com.google.protobuf.ByteString value) {
+      public Builder addKeyValueBytes(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -17530,9 +17530,9 @@ public final class AdminProtos {
        * repeated bytes key_value_bytes = 2;
        */
       public Builder addAllKeyValueBytes(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureKeyValueBytesIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, keyValueBytes_);
         onChanged();
         return this;
@@ -17601,12 +17601,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17624,22 +17624,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WALEntry parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WALEntry(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17651,7 +17651,7 @@ public final class AdminProtos {
 
   public interface ReplicateWALEntryRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicateWALEntryRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.WALEntry entry = 1;
@@ -17688,7 +17688,7 @@ public final class AdminProtos {
     /**
      * optional string replicationClusterId = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getReplicationClusterIdBytes();
 
     /**
@@ -17702,7 +17702,7 @@ public final class AdminProtos {
     /**
      * optional string sourceBaseNamespaceDirPath = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSourceBaseNamespaceDirPathBytes();
 
     /**
@@ -17716,7 +17716,7 @@ public final class AdminProtos {
     /**
      * optional string sourceHFileArchiveDirPath = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSourceHFileArchiveDirPathBytes();
   }
   /**
@@ -17730,11 +17730,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.ReplicateWALEntryRequest}
    */
   public  static final class ReplicateWALEntryRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicateWALEntryRequest)
       ReplicateWALEntryRequestOrBuilder {
     // Use ReplicateWALEntryRequest.newBuilder() to construct.
-    private ReplicateWALEntryRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicateWALEntryRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicateWALEntryRequest() {
@@ -17745,18 +17745,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicateWALEntryRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -17782,29 +17782,29 @@ public final class AdminProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               replicationClusterId_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               sourceBaseNamespaceDirPath_ = bs;
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               sourceHFileArchiveDirPath_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -17814,12 +17814,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -17878,8 +17878,8 @@ public final class AdminProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           replicationClusterId_ = s;
@@ -17890,17 +17890,17 @@ public final class AdminProtos {
     /**
      * optional string replicationClusterId = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getReplicationClusterIdBytes() {
       java.lang.Object ref = replicationClusterId_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         replicationClusterId_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -17920,8 +17920,8 @@ public final class AdminProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           sourceBaseNamespaceDirPath_ = s;
@@ -17932,17 +17932,17 @@ public final class AdminProtos {
     /**
      * optional string sourceBaseNamespaceDirPath = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSourceBaseNamespaceDirPathBytes() {
       java.lang.Object ref = sourceBaseNamespaceDirPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         sourceBaseNamespaceDirPath_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -17962,8 +17962,8 @@ public final class AdminProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           sourceHFileArchiveDirPath_ = s;
@@ -17974,17 +17974,17 @@ public final class AdminProtos {
     /**
      * optional string sourceHFileArchiveDirPath = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSourceHFileArchiveDirPathBytes() {
       java.lang.Object ref = sourceHFileArchiveDirPath_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         sourceHFileArchiveDirPath_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18004,19 +18004,19 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < entry_.size(); i++) {
         output.writeMessage(1, entry_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, replicationClusterId_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, replicationClusterId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, sourceBaseNamespaceDirPath_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, sourceBaseNamespaceDirPath_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, sourceHFileArchiveDirPath_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, sourceHFileArchiveDirPath_);
       }
       unknownFields.writeTo(output);
     }
@@ -18027,17 +18027,17 @@ public final class AdminProtos {
 
       size = 0;
       for (int i = 0; i < entry_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, entry_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationClusterId_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationClusterId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, sourceBaseNamespaceDirPath_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, sourceBaseNamespaceDirPath_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, sourceHFileArchiveDirPath_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, sourceHFileArchiveDirPath_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -18106,61 +18106,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -18178,7 +18178,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -18193,15 +18193,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.ReplicateWALEntryRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicateWALEntryRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -18214,12 +18214,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getEntryFieldBuilder();
         }
@@ -18241,7 +18241,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor;
       }
@@ -18292,29 +18292,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)other);
         } else {
@@ -18344,7 +18344,7 @@ public final class AdminProtos {
               entry_ = other.entry_;
               bitField0_ = (bitField0_ & ~0x00000001);
               entryBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getEntryFieldBuilder() : null;
             } else {
               entryBuilder_.addAllMessages(other.entry_);
@@ -18381,13 +18381,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -18408,7 +18408,7 @@ public final class AdminProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntryOrBuilder> entryBuilder_;
 
       /**
@@ -18540,7 +18540,7 @@ public final class AdminProtos {
           java.lang.Iterable values) {
         if (entryBuilder_ == null) {
           ensureEntryIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, entry_);
           onChanged();
         } else {
@@ -18624,11 +18624,11 @@ public final class AdminProtos {
            getEntryBuilderList() {
         return getEntryFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntryOrBuilder> 
           getEntryFieldBuilder() {
         if (entryBuilder_ == null) {
-          entryBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          entryBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntryOrBuilder>(
                   entry_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -18652,8 +18652,8 @@ public final class AdminProtos {
       public java.lang.String getReplicationClusterId() {
         java.lang.Object ref = replicationClusterId_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             replicationClusterId_ = s;
@@ -18666,17 +18666,17 @@ public final class AdminProtos {
       /**
        * optional string replicationClusterId = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getReplicationClusterIdBytes() {
         java.lang.Object ref = replicationClusterId_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           replicationClusterId_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -18705,7 +18705,7 @@ public final class AdminProtos {
        * optional string replicationClusterId = 2;
        */
       public Builder setReplicationClusterIdBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -18728,8 +18728,8 @@ public final class AdminProtos {
       public java.lang.String getSourceBaseNamespaceDirPath() {
         java.lang.Object ref = sourceBaseNamespaceDirPath_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             sourceBaseNamespaceDirPath_ = s;
@@ -18742,17 +18742,17 @@ public final class AdminProtos {
       /**
        * optional string sourceBaseNamespaceDirPath = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getSourceBaseNamespaceDirPathBytes() {
         java.lang.Object ref = sourceBaseNamespaceDirPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           sourceBaseNamespaceDirPath_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -18781,7 +18781,7 @@ public final class AdminProtos {
        * optional string sourceBaseNamespaceDirPath = 3;
        */
       public Builder setSourceBaseNamespaceDirPathBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -18804,8 +18804,8 @@ public final class AdminProtos {
       public java.lang.String getSourceHFileArchiveDirPath() {
         java.lang.Object ref = sourceHFileArchiveDirPath_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             sourceHFileArchiveDirPath_ = s;
@@ -18818,17 +18818,17 @@ public final class AdminProtos {
       /**
        * optional string sourceHFileArchiveDirPath = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getSourceHFileArchiveDirPathBytes() {
         java.lang.Object ref = sourceHFileArchiveDirPath_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           sourceHFileArchiveDirPath_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -18857,7 +18857,7 @@ public final class AdminProtos {
        * optional string sourceHFileArchiveDirPath = 4;
        */
       public Builder setSourceHFileArchiveDirPathBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -18867,12 +18867,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -18890,22 +18890,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicateWALEntryRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicateWALEntryRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -18917,34 +18917,34 @@ public final class AdminProtos {
 
   public interface ReplicateWALEntryResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicateWALEntryResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.ReplicateWALEntryResponse}
    */
   public  static final class ReplicateWALEntryResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicateWALEntryResponse)
       ReplicateWALEntryResponseOrBuilder {
     // Use ReplicateWALEntryResponse.newBuilder() to construct.
-    private ReplicateWALEntryResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicateWALEntryResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicateWALEntryResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicateWALEntryResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -18962,22 +18962,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -18994,7 +18994,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -19038,61 +19038,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -19110,7 +19110,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -19118,15 +19118,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.ReplicateWALEntryResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicateWALEntryResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -19139,12 +19139,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -19153,7 +19153,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor;
       }
@@ -19180,29 +19180,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse)other);
         } else {
@@ -19223,13 +19223,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -19240,12 +19240,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -19263,22 +19263,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicateWALEntryResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicateWALEntryResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -19290,34 +19290,34 @@ public final class AdminProtos {
 
   public interface RollWALWriterRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RollWALWriterRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.RollWALWriterRequest}
    */
   public  static final class RollWALWriterRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RollWALWriterRequest)
       RollWALWriterRequestOrBuilder {
     // Use RollWALWriterRequest.newBuilder() to construct.
-    private RollWALWriterRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RollWALWriterRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RollWALWriterRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RollWALWriterRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19335,22 +19335,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19367,7 +19367,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -19411,61 +19411,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -19483,7 +19483,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -19491,15 +19491,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.RollWALWriterRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RollWALWriterRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -19512,12 +19512,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -19526,7 +19526,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterRequest_descriptor;
       }
@@ -19553,29 +19553,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest)other);
         } else {
@@ -19596,13 +19596,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -19613,12 +19613,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -19636,22 +19636,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RollWALWriterRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RollWALWriterRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -19663,7 +19663,7 @@ public final class AdminProtos {
 
   public interface RollWALWriterResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RollWALWriterResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -19672,7 +19672,7 @@ public final class AdminProtos {
      *
      * repeated bytes region_to_flush = 1;
      */
-    java.util.List getRegionToFlushList();
+    java.util.List getRegionToFlushList();
     /**
      * 
      * A list of encoded name of regions to flush
@@ -19688,7 +19688,7 @@ public final class AdminProtos {
      *
      * repeated bytes region_to_flush = 1;
      */
-    com.google.protobuf.ByteString getRegionToFlush(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionToFlush(int index);
   }
   /**
    * 
@@ -19699,11 +19699,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.RollWALWriterResponse}
    */
   public  static final class RollWALWriterResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RollWALWriterResponse)
       RollWALWriterResponseOrBuilder {
     // Use RollWALWriterResponse.newBuilder() to construct.
-    private RollWALWriterResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RollWALWriterResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RollWALWriterResponse() {
@@ -19711,18 +19711,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RollWALWriterResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19740,7 +19740,7 @@ public final class AdminProtos {
             }
             case 10: {
               if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                regionToFlush_ = new java.util.ArrayList();
+                regionToFlush_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000001;
               }
               regionToFlush_.add(input.readBytes());
@@ -19748,10 +19748,10 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -19761,12 +19761,12 @@ public final class AdminProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19774,7 +19774,7 @@ public final class AdminProtos {
     }
 
     public static final int REGION_TO_FLUSH_FIELD_NUMBER = 1;
-    private java.util.List regionToFlush_;
+    private java.util.List regionToFlush_;
     /**
      * 
      * A list of encoded name of regions to flush
@@ -19782,7 +19782,7 @@ public final class AdminProtos {
      *
      * repeated bytes region_to_flush = 1;
      */
-    public java.util.List
+    public java.util.List
         getRegionToFlushList() {
       return regionToFlush_;
     }
@@ -19803,7 +19803,7 @@ public final class AdminProtos {
      *
      * repeated bytes region_to_flush = 1;
      */
-    public com.google.protobuf.ByteString getRegionToFlush(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionToFlush(int index) {
       return regionToFlush_.get(index);
     }
 
@@ -19817,7 +19817,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < regionToFlush_.size(); i++) {
         output.writeBytes(1, regionToFlush_.get(i));
@@ -19833,7 +19833,7 @@ public final class AdminProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < regionToFlush_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(regionToFlush_.get(i));
         }
         size += dataSize;
@@ -19879,61 +19879,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -19951,7 +19951,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -19964,15 +19964,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.RollWALWriterResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RollWALWriterResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -19985,12 +19985,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -20001,7 +20001,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_RollWALWriterResponse_descriptor;
       }
@@ -20034,29 +20034,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse)other);
         } else {
@@ -20087,13 +20087,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20105,10 +20105,10 @@ public final class AdminProtos {
       }
       private int bitField0_;
 
-      private java.util.List regionToFlush_ = java.util.Collections.emptyList();
+      private java.util.List regionToFlush_ = java.util.Collections.emptyList();
       private void ensureRegionToFlushIsMutable() {
         if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          regionToFlush_ = new java.util.ArrayList(regionToFlush_);
+          regionToFlush_ = new java.util.ArrayList(regionToFlush_);
           bitField0_ |= 0x00000001;
          }
       }
@@ -20119,7 +20119,7 @@ public final class AdminProtos {
        *
        * repeated bytes region_to_flush = 1;
        */
-      public java.util.List
+      public java.util.List
           getRegionToFlushList() {
         return java.util.Collections.unmodifiableList(regionToFlush_);
       }
@@ -20140,7 +20140,7 @@ public final class AdminProtos {
        *
        * repeated bytes region_to_flush = 1;
        */
-      public com.google.protobuf.ByteString getRegionToFlush(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionToFlush(int index) {
         return regionToFlush_.get(index);
       }
       /**
@@ -20151,7 +20151,7 @@ public final class AdminProtos {
        * repeated bytes region_to_flush = 1;
        */
       public Builder setRegionToFlush(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -20167,7 +20167,7 @@ public final class AdminProtos {
        *
        * repeated bytes region_to_flush = 1;
        */
-      public Builder addRegionToFlush(com.google.protobuf.ByteString value) {
+      public Builder addRegionToFlush(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -20184,9 +20184,9 @@ public final class AdminProtos {
        * repeated bytes region_to_flush = 1;
        */
       public Builder addAllRegionToFlush(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureRegionToFlushIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, regionToFlush_);
         onChanged();
         return this;
@@ -20205,12 +20205,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -20228,22 +20228,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RollWALWriterResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RollWALWriterResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -20255,7 +20255,7 @@ public final class AdminProtos {
 
   public interface StopServerRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StopServerRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string reason = 1;
@@ -20268,18 +20268,18 @@ public final class AdminProtos {
     /**
      * required string reason = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getReasonBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.StopServerRequest}
    */
   public  static final class StopServerRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StopServerRequest)
       StopServerRequestOrBuilder {
     // Use StopServerRequest.newBuilder() to construct.
-    private StopServerRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StopServerRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StopServerRequest() {
@@ -20287,18 +20287,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StopServerRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -20315,29 +20315,29 @@ public final class AdminProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               reason_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -20361,8 +20361,8 @@ public final class AdminProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           reason_ = s;
@@ -20373,17 +20373,17 @@ public final class AdminProtos {
     /**
      * required string reason = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getReasonBytes() {
       java.lang.Object ref = reason_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         reason_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -20401,10 +20401,10 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, reason_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, reason_);
       }
       unknownFields.writeTo(output);
     }
@@ -20415,7 +20415,7 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, reason_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, reason_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -20460,61 +20460,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -20532,7 +20532,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -20540,15 +20540,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.StopServerRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StopServerRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -20561,12 +20561,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -20577,7 +20577,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerRequest_descriptor;
       }
@@ -20611,29 +20611,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest)other);
         } else {
@@ -20662,13 +20662,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20693,8 +20693,8 @@ public final class AdminProtos {
       public java.lang.String getReason() {
         java.lang.Object ref = reason_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             reason_ = s;
@@ -20707,17 +20707,17 @@ public final class AdminProtos {
       /**
        * required string reason = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getReasonBytes() {
         java.lang.Object ref = reason_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           reason_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -20746,7 +20746,7 @@ public final class AdminProtos {
        * required string reason = 1;
        */
       public Builder setReasonBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -20756,12 +20756,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -20779,22 +20779,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StopServerRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StopServerRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -20806,34 +20806,34 @@ public final class AdminProtos {
 
   public interface StopServerResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StopServerResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.StopServerResponse}
    */
   public  static final class StopServerResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StopServerResponse)
       StopServerResponseOrBuilder {
     // Use StopServerResponse.newBuilder() to construct.
-    private StopServerResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StopServerResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StopServerResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StopServerResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -20851,22 +20851,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -20883,7 +20883,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -20927,61 +20927,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -20999,7 +20999,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -21007,15 +21007,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.StopServerResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StopServerResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21028,12 +21028,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -21042,7 +21042,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_StopServerResponse_descriptor;
       }
@@ -21069,29 +21069,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse)other);
         } else {
@@ -21112,13 +21112,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -21129,12 +21129,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -21152,22 +21152,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StopServerResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StopServerResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -21179,34 +21179,34 @@ public final class AdminProtos {
 
   public interface GetServerInfoRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetServerInfoRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.GetServerInfoRequest}
    */
   public  static final class GetServerInfoRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetServerInfoRequest)
       GetServerInfoRequestOrBuilder {
     // Use GetServerInfoRequest.newBuilder() to construct.
-    private GetServerInfoRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetServerInfoRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetServerInfoRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetServerInfoRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -21224,22 +21224,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -21256,7 +21256,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -21300,61 +21300,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -21372,7 +21372,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -21380,15 +21380,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetServerInfoRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetServerInfoRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21401,12 +21401,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -21415,7 +21415,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoRequest_descriptor;
       }
@@ -21442,29 +21442,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest)other);
         } else {
@@ -21485,13 +21485,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -21502,12 +21502,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -21525,22 +21525,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetServerInfoRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetServerInfoRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -21552,7 +21552,7 @@ public final class AdminProtos {
 
   public interface ServerInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ServerInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ServerName server_name = 1;
@@ -21580,11 +21580,11 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.ServerInfo}
    */
   public  static final class ServerInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ServerInfo)
       ServerInfoOrBuilder {
     // Use ServerInfo.newBuilder() to construct.
-    private ServerInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ServerInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ServerInfo() {
@@ -21592,18 +21592,18 @@ public final class AdminProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ServerInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -21639,22 +21639,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ServerInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ServerInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -21716,7 +21716,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServerName());
@@ -21733,11 +21733,11 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServerName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, webuiPort_);
       }
       size += unknownFields.getSerializedSize();
@@ -21792,61 +21792,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -21864,7 +21864,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -21872,15 +21872,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.ServerInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ServerInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ServerInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ServerInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21893,12 +21893,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerNameFieldBuilder();
         }
@@ -21916,7 +21916,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_ServerInfo_descriptor;
       }
@@ -21958,29 +21958,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo)other);
         } else {
@@ -22013,13 +22013,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -22032,7 +22032,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_;
       /**
        * required .hbase.pb.ServerName server_name = 1;
@@ -22135,11 +22135,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.ServerName server_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerNameFieldBuilder() {
         if (serverNameBuilder_ == null) {
-          serverNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServerName(),
                   getParentForChildren(),
@@ -22181,12 +22181,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -22204,22 +22204,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ServerInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ServerInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -22231,7 +22231,7 @@ public final class AdminProtos {
 
   public interface GetServerInfoResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetServerInfoResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ServerInfo server_info = 1;
@@ -22250,29 +22250,29 @@ public final class AdminProtos {
    * Protobuf type {@code hbase.pb.GetServerInfoResponse}
    */
   public  static final class GetServerInfoResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetServerInfoResponse)
       GetServerInfoResponseOrBuilder {
     // Use GetServerInfoResponse.newBuilder() to construct.
-    private GetServerInfoResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetServerInfoResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetServerInfoResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetServerInfoResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -22303,22 +22303,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -22365,7 +22365,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServerInfo());
@@ -22379,7 +22379,7 @@ public final class AdminProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServerInfo());
       }
       size += unknownFields.getSerializedSize();
@@ -22425,61 +22425,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -22497,7 +22497,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -22505,15 +22505,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.GetServerInfoResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetServerInfoResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -22526,12 +22526,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerInfoFieldBuilder();
         }
@@ -22547,7 +22547,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_GetServerInfoResponse_descriptor;
       }
@@ -22585,29 +22585,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse)other);
         } else {
@@ -22637,13 +22637,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -22656,7 +22656,7 @@ public final class AdminProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo serverInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfoOrBuilder> serverInfoBuilder_;
       /**
        * required .hbase.pb.ServerInfo server_info = 1;
@@ -22759,11 +22759,11 @@ public final class AdminProtos {
       /**
        * required .hbase.pb.ServerInfo server_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfoOrBuilder> 
           getServerInfoFieldBuilder() {
         if (serverInfoBuilder_ == null) {
-          serverInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfoOrBuilder>(
                   getServerInfo(),
                   getParentForChildren(),
@@ -22773,12 +22773,12 @@ public final class AdminProtos {
         return serverInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -22796,22 +22796,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetServerInfoResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetServerInfoResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -22823,34 +22823,34 @@ public final class AdminProtos {
 
   public interface UpdateConfigurationRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UpdateConfigurationRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.UpdateConfigurationRequest}
    */
   public  static final class UpdateConfigurationRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UpdateConfigurationRequest)
       UpdateConfigurationRequestOrBuilder {
     // Use UpdateConfigurationRequest.newBuilder() to construct.
-    private UpdateConfigurationRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UpdateConfigurationRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UpdateConfigurationRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UpdateConfigurationRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -22868,22 +22868,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -22900,7 +22900,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -22944,61 +22944,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -23016,7 +23016,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -23024,15 +23024,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.UpdateConfigurationRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UpdateConfigurationRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -23045,12 +23045,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -23059,7 +23059,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationRequest_descriptor;
       }
@@ -23086,29 +23086,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest)other);
         } else {
@@ -23129,13 +23129,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -23146,12 +23146,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -23169,22 +23169,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UpdateConfigurationRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UpdateConfigurationRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -23196,34 +23196,34 @@ public final class AdminProtos {
 
   public interface UpdateConfigurationResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UpdateConfigurationResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.UpdateConfigurationResponse}
    */
   public  static final class UpdateConfigurationResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UpdateConfigurationResponse)
       UpdateConfigurationResponseOrBuilder {
     // Use UpdateConfigurationResponse.newBuilder() to construct.
-    private UpdateConfigurationResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UpdateConfigurationResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UpdateConfigurationResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UpdateConfigurationResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -23241,22 +23241,22 @@ public final class AdminProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -23273,7 +23273,7 @@ public final class AdminProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -23317,61 +23317,61 @@ public final class AdminProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -23389,7 +23389,7 @@ public final class AdminProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -23397,15 +23397,15 @@ public final class AdminProtos {
      * Protobuf type {@code hbase.pb.UpdateConfigurationResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UpdateConfigurationResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -23418,12 +23418,12 @@ public final class AdminProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -23432,7 +23432,7 @@ public final class AdminProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_UpdateConfigurationResponse_descriptor;
       }
@@ -23459,29 +23459,29 @@ public final class AdminProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse)other);
         } else {
@@ -23502,13 +23502,13 @@ public final class AdminProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -23519,12 +23519,12 @@ public final class AdminProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -23542,22 +23542,22 @@ public final class AdminProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UpdateConfigurationResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UpdateConfigurationResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -23571,7 +23571,7 @@ public final class AdminProtos {
    * Protobuf service {@code hbase.pb.AdminService}
    */
   public static abstract class AdminService
-      implements com.google.protobuf.Service {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.Service {
     protected AdminService() {}
 
     public interface Interface {
@@ -23579,295 +23579,295 @@ public final class AdminProtos {
        * rpc GetRegionInfo(.hbase.pb.GetRegionInfoRequest) returns (.hbase.pb.GetRegionInfoResponse);
        */
       public abstract void getRegionInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc GetStoreFile(.hbase.pb.GetStoreFileRequest) returns (.hbase.pb.GetStoreFileResponse);
        */
       public abstract void getStoreFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc GetOnlineRegion(.hbase.pb.GetOnlineRegionRequest) returns (.hbase.pb.GetOnlineRegionResponse);
        */
       public abstract void getOnlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc OpenRegion(.hbase.pb.OpenRegionRequest) returns (.hbase.pb.OpenRegionResponse);
        */
       public abstract void openRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc WarmupRegion(.hbase.pb.WarmupRegionRequest) returns (.hbase.pb.WarmupRegionResponse);
        */
       public abstract void warmupRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc CloseRegion(.hbase.pb.CloseRegionRequest) returns (.hbase.pb.CloseRegionResponse);
        */
       public abstract void closeRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc FlushRegion(.hbase.pb.FlushRegionRequest) returns (.hbase.pb.FlushRegionResponse);
        */
       public abstract void flushRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc SplitRegion(.hbase.pb.SplitRegionRequest) returns (.hbase.pb.SplitRegionResponse);
        */
       public abstract void splitRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc CompactRegion(.hbase.pb.CompactRegionRequest) returns (.hbase.pb.CompactRegionResponse);
        */
       public abstract void compactRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc MergeRegions(.hbase.pb.MergeRegionsRequest) returns (.hbase.pb.MergeRegionsResponse);
        */
       public abstract void mergeRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc ReplicateWALEntry(.hbase.pb.ReplicateWALEntryRequest) returns (.hbase.pb.ReplicateWALEntryResponse);
        */
       public abstract void replicateWALEntry(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc Replay(.hbase.pb.ReplicateWALEntryRequest) returns (.hbase.pb.ReplicateWALEntryResponse);
        */
       public abstract void replay(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc RollWALWriter(.hbase.pb.RollWALWriterRequest) returns (.hbase.pb.RollWALWriterResponse);
        */
       public abstract void rollWALWriter(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc GetServerInfo(.hbase.pb.GetServerInfoRequest) returns (.hbase.pb.GetServerInfoResponse);
        */
       public abstract void getServerInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc StopServer(.hbase.pb.StopServerRequest) returns (.hbase.pb.StopServerResponse);
        */
       public abstract void stopServer(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc UpdateFavoredNodes(.hbase.pb.UpdateFavoredNodesRequest) returns (.hbase.pb.UpdateFavoredNodesResponse);
        */
       public abstract void updateFavoredNodes(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc UpdateConfiguration(.hbase.pb.UpdateConfigurationRequest) returns (.hbase.pb.UpdateConfigurationResponse);
        */
       public abstract void updateConfiguration(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     }
 
-    public static com.google.protobuf.Service newReflectiveService(
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
         final Interface impl) {
       return new AdminService() {
         @java.lang.Override
         public  void getRegionInfo(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getRegionInfo(controller, request, done);
         }
 
         @java.lang.Override
         public  void getStoreFile(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getStoreFile(controller, request, done);
         }
 
         @java.lang.Override
         public  void getOnlineRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getOnlineRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void openRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.openRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void warmupRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.warmupRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void closeRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.closeRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void flushRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.flushRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void splitRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.splitRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void compactRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.compactRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void mergeRegions(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.mergeRegions(controller, request, done);
         }
 
         @java.lang.Override
         public  void replicateWALEntry(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.replicateWALEntry(controller, request, done);
         }
 
         @java.lang.Override
         public  void replay(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.replay(controller, request, done);
         }
 
         @java.lang.Override
         public  void rollWALWriter(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.rollWALWriter(controller, request, done);
         }
 
         @java.lang.Override
         public  void getServerInfo(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getServerInfo(controller, request, done);
         }
 
         @java.lang.Override
         public  void stopServer(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.stopServer(controller, request, done);
         }
 
         @java.lang.Override
         public  void updateFavoredNodes(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.updateFavoredNodes(controller, request, done);
         }
 
         @java.lang.Override
         public  void updateConfiguration(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.updateConfiguration(controller, request, done);
         }
 
       };
     }
 
-    public static com.google.protobuf.BlockingService
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
         newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService() {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
             getDescriptorForType() {
           return getDescriptor();
         }
 
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message callBlockingMethod(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.callBlockingMethod() given method descriptor for " +
@@ -23913,9 +23913,9 @@ public final class AdminProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getRequestPrototype() given method " +
@@ -23961,9 +23961,9 @@ public final class AdminProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getResponsePrototype() given method " +
@@ -24016,154 +24016,154 @@ public final class AdminProtos {
      * rpc GetRegionInfo(.hbase.pb.GetRegionInfoRequest) returns (.hbase.pb.GetRegionInfoResponse);
      */
     public abstract void getRegionInfo(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc GetStoreFile(.hbase.pb.GetStoreFileRequest) returns (.hbase.pb.GetStoreFileResponse);
      */
     public abstract void getStoreFile(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc GetOnlineRegion(.hbase.pb.GetOnlineRegionRequest) returns (.hbase.pb.GetOnlineRegionResponse);
      */
     public abstract void getOnlineRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc OpenRegion(.hbase.pb.OpenRegionRequest) returns (.hbase.pb.OpenRegionResponse);
      */
     public abstract void openRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc WarmupRegion(.hbase.pb.WarmupRegionRequest) returns (.hbase.pb.WarmupRegionResponse);
      */
     public abstract void warmupRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc CloseRegion(.hbase.pb.CloseRegionRequest) returns (.hbase.pb.CloseRegionResponse);
      */
     public abstract void closeRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc FlushRegion(.hbase.pb.FlushRegionRequest) returns (.hbase.pb.FlushRegionResponse);
      */
     public abstract void flushRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc SplitRegion(.hbase.pb.SplitRegionRequest) returns (.hbase.pb.SplitRegionResponse);
      */
     public abstract void splitRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc CompactRegion(.hbase.pb.CompactRegionRequest) returns (.hbase.pb.CompactRegionResponse);
      */
     public abstract void compactRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc MergeRegions(.hbase.pb.MergeRegionsRequest) returns (.hbase.pb.MergeRegionsResponse);
      */
     public abstract void mergeRegions(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc ReplicateWALEntry(.hbase.pb.ReplicateWALEntryRequest) returns (.hbase.pb.ReplicateWALEntryResponse);
      */
     public abstract void replicateWALEntry(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc Replay(.hbase.pb.ReplicateWALEntryRequest) returns (.hbase.pb.ReplicateWALEntryResponse);
      */
     public abstract void replay(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc RollWALWriter(.hbase.pb.RollWALWriterRequest) returns (.hbase.pb.RollWALWriterResponse);
      */
     public abstract void rollWALWriter(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc GetServerInfo(.hbase.pb.GetServerInfoRequest) returns (.hbase.pb.GetServerInfoResponse);
      */
     public abstract void getServerInfo(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc StopServer(.hbase.pb.StopServerRequest) returns (.hbase.pb.StopServerResponse);
      */
     public abstract void stopServer(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc UpdateFavoredNodes(.hbase.pb.UpdateFavoredNodesRequest) returns (.hbase.pb.UpdateFavoredNodesResponse);
      */
     public abstract void updateFavoredNodes(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc UpdateConfiguration(.hbase.pb.UpdateConfigurationRequest) returns (.hbase.pb.UpdateConfigurationResponse);
      */
     public abstract void updateConfiguration(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.getDescriptor().getServices().get(0);
     }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
 
     public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Message> done) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.callMethod() given method descriptor for wrong " +
@@ -24172,87 +24172,87 @@ public final class AdminProtos {
       switch(method.getIndex()) {
         case 0:
           this.getRegionInfo(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 1:
           this.getStoreFile(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 2:
           this.getOnlineRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 3:
           this.openRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 4:
           this.warmupRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 5:
           this.closeRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 6:
           this.flushRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 7:
           this.splitRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 8:
           this.compactRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 9:
           this.mergeRegions(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 10:
           this.replicateWALEntry(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 11:
           this.replay(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 12:
           this.rollWALWriter(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 13:
           this.getServerInfo(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 14:
           this.stopServer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 15:
           this.updateFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 16:
           this.updateConfiguration(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         default:
@@ -24260,9 +24260,9 @@ public final class AdminProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getRequestPrototype() given method " +
@@ -24308,9 +24308,9 @@ public final class AdminProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getResponsePrototype() given method " +
@@ -24357,271 +24357,271 @@ public final class AdminProtos {
     }
 
     public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
       return new Stub(channel);
     }
 
     public static final class Stub extends org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
+      private Stub(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.RpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel;
 
-      public com.google.protobuf.RpcChannel getChannel() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel getChannel() {
         return channel;
       }
 
       public  void getRegionInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(0),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.getDefaultInstance()));
       }
 
       public  void getStoreFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse.getDefaultInstance()));
       }
 
       public  void getOnlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(2),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse.getDefaultInstance()));
       }
 
       public  void openRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(3),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.getDefaultInstance()));
       }
 
       public  void warmupRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(4),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance()));
       }
 
       public  void closeRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(5),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance()));
       }
 
       public  void flushRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(6),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance()));
       }
 
       public  void splitRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(7),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance()));
       }
 
       public  void compactRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(8),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance()));
       }
 
       public  void mergeRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(9),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance()));
       }
 
       public  void replicateWALEntry(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(10),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance()));
       }
 
       public  void replay(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(11),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance()));
       }
 
       public  void rollWALWriter(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(12),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance()));
       }
 
       public  void getServerInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(13),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance()));
       }
 
       public  void stopServer(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(14),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance()));
       }
 
       public  void updateFavoredNodes(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(15),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance()));
       }
 
       public  void updateConfiguration(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(16),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance()));
@@ -24629,108 +24629,108 @@ public final class AdminProtos {
     }
 
     public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
       return new BlockingStub(channel);
     }
 
     public interface BlockingInterface {
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse getRegionInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse getStoreFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse getOnlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse openRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse warmupRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse closeRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse flushRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse splitRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse compactRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse mergeRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replicateWALEntry(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replay(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse getServerInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse stopServer(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse updateFavoredNodes(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse updateConfiguration(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+      private BlockingStub(org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.BlockingRpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse getRegionInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(0),
           controller,
@@ -24740,9 +24740,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse getStoreFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(1),
           controller,
@@ -24752,9 +24752,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse getOnlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(2),
           controller,
@@ -24764,9 +24764,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse openRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(3),
           controller,
@@ -24776,9 +24776,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse warmupRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(4),
           controller,
@@ -24788,9 +24788,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse closeRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(5),
           controller,
@@ -24800,9 +24800,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse flushRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(6),
           controller,
@@ -24812,9 +24812,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse splitRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(7),
           controller,
@@ -24824,9 +24824,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse compactRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(8),
           controller,
@@ -24836,9 +24836,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse mergeRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(9),
           controller,
@@ -24848,9 +24848,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replicateWALEntry(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(10),
           controller,
@@ -24860,9 +24860,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse replay(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(11),
           controller,
@@ -24872,9 +24872,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(12),
           controller,
@@ -24884,9 +24884,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse getServerInfo(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(13),
           controller,
@@ -24896,9 +24896,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse stopServer(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(14),
           controller,
@@ -24908,9 +24908,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse updateFavoredNodes(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(15),
           controller,
@@ -24920,9 +24920,9 @@ public final class AdminProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse updateConfiguration(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(16),
           controller,
@@ -24935,192 +24935,192 @@ public final class AdminProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.AdminService)
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetRegionInfoRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetRegionInfoRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetRegionInfoResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetRegionInfoResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetStoreFileRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetStoreFileRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetStoreFileResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetStoreFileResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetOnlineRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetOnlineRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetOnlineRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetOnlineRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_OpenRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_OpenRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_OpenRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_OpenRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WarmupRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WarmupRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WarmupRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WarmupRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CloseRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CloseRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CloseRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CloseRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FlushRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FlushRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FlushRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FlushRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SplitRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SplitRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SplitRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SplitRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CompactRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CompactRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CompactRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CompactRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UpdateFavoredNodesRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UpdateFavoredNodesResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UpdateFavoredNodesResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MergeRegionsRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MergeRegionsRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MergeRegionsResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MergeRegionsResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WALEntry_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WALEntry_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicateWALEntryRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicateWALEntryResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RollWALWriterRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RollWALWriterRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RollWALWriterResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RollWALWriterResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StopServerRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StopServerRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StopServerResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StopServerResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetServerInfoRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetServerInfoRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ServerInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ServerInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetServerInfoResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetServerInfoResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UpdateConfigurationRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UpdateConfigurationRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UpdateConfigurationResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UpdateConfigurationResponse_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -25237,234 +25237,234 @@ public final class AdminProtos {
       ".hadoop.hbase.shaded.protobuf.generatedB" +
       "\013AdminProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_GetRegionInfoRequest_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_GetRegionInfoRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetRegionInfoRequest_descriptor,
         new java.lang.String[] { "Region", "CompactionState", });
     internal_static_hbase_pb_GetRegionInfoResponse_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_GetRegionInfoResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetRegionInfoResponse_descriptor,
         new java.lang.String[] { "RegionInfo", "CompactionState", "IsRecovering", });
     internal_static_hbase_pb_GetStoreFileRequest_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_GetStoreFileRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetStoreFileRequest_descriptor,
         new java.lang.String[] { "Region", "Family", });
     internal_static_hbase_pb_GetStoreFileResponse_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_GetStoreFileResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetStoreFileResponse_descriptor,
         new java.lang.String[] { "StoreFile", });
     internal_static_hbase_pb_GetOnlineRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_GetOnlineRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetOnlineRegionRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_GetOnlineRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_GetOnlineRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetOnlineRegionResponse_descriptor,
         new java.lang.String[] { "RegionInfo", });
     internal_static_hbase_pb_OpenRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_OpenRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_OpenRegionRequest_descriptor,
         new java.lang.String[] { "OpenInfo", "ServerStartCode", "MasterSystemTime", });
     internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_descriptor =
       internal_static_hbase_pb_OpenRegionRequest_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_OpenRegionRequest_RegionOpenInfo_descriptor,
         new java.lang.String[] { "Region", "VersionOfOfflineNode", "FavoredNodes", "OpenForDistributedLogReplay", });
     internal_static_hbase_pb_OpenRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_OpenRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_OpenRegionResponse_descriptor,
         new java.lang.String[] { "OpeningState", });
     internal_static_hbase_pb_WarmupRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_WarmupRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WarmupRegionRequest_descriptor,
         new java.lang.String[] { "RegionInfo", });
     internal_static_hbase_pb_WarmupRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_WarmupRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WarmupRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_CloseRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_CloseRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CloseRegionRequest_descriptor,
         new java.lang.String[] { "Region", "VersionOfClosingNode", "TransitionInZK", "DestinationServer", "ServerStartCode", });
     internal_static_hbase_pb_CloseRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(11);
     internal_static_hbase_pb_CloseRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CloseRegionResponse_descriptor,
         new java.lang.String[] { "Closed", });
     internal_static_hbase_pb_FlushRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_FlushRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FlushRegionRequest_descriptor,
         new java.lang.String[] { "Region", "IfOlderThanTs", "WriteFlushWalMarker", });
     internal_static_hbase_pb_FlushRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_hbase_pb_FlushRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FlushRegionResponse_descriptor,
         new java.lang.String[] { "LastFlushTime", "Flushed", "WroteFlushWalMarker", });
     internal_static_hbase_pb_SplitRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_SplitRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SplitRegionRequest_descriptor,
         new java.lang.String[] { "Region", "SplitPoint", });
     internal_static_hbase_pb_SplitRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(15);
     internal_static_hbase_pb_SplitRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SplitRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_CompactRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(16);
     internal_static_hbase_pb_CompactRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CompactRegionRequest_descriptor,
         new java.lang.String[] { "Region", "Major", "Family", });
     internal_static_hbase_pb_CompactRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(17);
     internal_static_hbase_pb_CompactRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CompactRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor =
       getDescriptor().getMessageTypes().get(18);
     internal_static_hbase_pb_UpdateFavoredNodesRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor,
         new java.lang.String[] { "UpdateInfo", });
     internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor =
       internal_static_hbase_pb_UpdateFavoredNodesRequest_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor,
         new java.lang.String[] { "Region", "FavoredNodes", });
     internal_static_hbase_pb_UpdateFavoredNodesResponse_descriptor =
       getDescriptor().getMessageTypes().get(19);
     internal_static_hbase_pb_UpdateFavoredNodesResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UpdateFavoredNodesResponse_descriptor,
         new java.lang.String[] { "Response", });
     internal_static_hbase_pb_MergeRegionsRequest_descriptor =
       getDescriptor().getMessageTypes().get(20);
     internal_static_hbase_pb_MergeRegionsRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MergeRegionsRequest_descriptor,
         new java.lang.String[] { "RegionA", "RegionB", "Forcible", "MasterSystemTime", });
     internal_static_hbase_pb_MergeRegionsResponse_descriptor =
       getDescriptor().getMessageTypes().get(21);
     internal_static_hbase_pb_MergeRegionsResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MergeRegionsResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_WALEntry_descriptor =
       getDescriptor().getMessageTypes().get(22);
     internal_static_hbase_pb_WALEntry_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WALEntry_descriptor,
         new java.lang.String[] { "Key", "KeyValueBytes", "AssociatedCellCount", });
     internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor =
       getDescriptor().getMessageTypes().get(23);
     internal_static_hbase_pb_ReplicateWALEntryRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor,
         new java.lang.String[] { "Entry", "ReplicationClusterId", "SourceBaseNamespaceDirPath", "SourceHFileArchiveDirPath", });
     internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor =
       getDescriptor().getMessageTypes().get(24);
     internal_static_hbase_pb_ReplicateWALEntryResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_RollWALWriterRequest_descriptor =
       getDescriptor().getMessageTypes().get(25);
     internal_static_hbase_pb_RollWALWriterRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RollWALWriterRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_RollWALWriterResponse_descriptor =
       getDescriptor().getMessageTypes().get(26);
     internal_static_hbase_pb_RollWALWriterResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RollWALWriterResponse_descriptor,
         new java.lang.String[] { "RegionToFlush", });
     internal_static_hbase_pb_StopServerRequest_descriptor =
       getDescriptor().getMessageTypes().get(27);
     internal_static_hbase_pb_StopServerRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StopServerRequest_descriptor,
         new java.lang.String[] { "Reason", });
     internal_static_hbase_pb_StopServerResponse_descriptor =
       getDescriptor().getMessageTypes().get(28);
     internal_static_hbase_pb_StopServerResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StopServerResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_GetServerInfoRequest_descriptor =
       getDescriptor().getMessageTypes().get(29);
     internal_static_hbase_pb_GetServerInfoRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetServerInfoRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_ServerInfo_descriptor =
       getDescriptor().getMessageTypes().get(30);
     internal_static_hbase_pb_ServerInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ServerInfo_descriptor,
         new java.lang.String[] { "ServerName", "WebuiPort", });
     internal_static_hbase_pb_GetServerInfoResponse_descriptor =
       getDescriptor().getMessageTypes().get(31);
     internal_static_hbase_pb_GetServerInfoResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetServerInfoResponse_descriptor,
         new java.lang.String[] { "ServerInfo", });
     internal_static_hbase_pb_UpdateConfigurationRequest_descriptor =
       getDescriptor().getMessageTypes().get(32);
     internal_static_hbase_pb_UpdateConfigurationRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UpdateConfigurationRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_UpdateConfigurationResponse_descriptor =
       getDescriptor().getMessageTypes().get(33);
     internal_static_hbase_pb_UpdateConfigurationResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UpdateConfigurationResponse_descriptor,
         new java.lang.String[] { });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
index f049d50..60ca467 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
@@ -6,13 +6,13 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class CellProtos {
   private CellProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * 
@@ -23,7 +23,7 @@ public final class CellProtos {
    * Protobuf enum {@code hbase.pb.CellType}
    */
   public enum CellType
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * MINIMUM = 0;
      */
@@ -108,27 +108,27 @@ public final class CellProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         CellType> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public CellType findValueByNumber(int number) {
               return CellType.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -136,7 +136,7 @@ public final class CellProtos {
     private static final CellType[] VALUES = values();
 
     public static CellType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -155,7 +155,7 @@ public final class CellProtos {
 
   public interface CellOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Cell)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes row = 1;
@@ -164,7 +164,7 @@ public final class CellProtos {
     /**
      * optional bytes row = 1;
      */
-    com.google.protobuf.ByteString getRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow();
 
     /**
      * optional bytes family = 2;
@@ -173,7 +173,7 @@ public final class CellProtos {
     /**
      * optional bytes family = 2;
      */
-    com.google.protobuf.ByteString getFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
     /**
      * optional bytes qualifier = 3;
@@ -182,7 +182,7 @@ public final class CellProtos {
     /**
      * optional bytes qualifier = 3;
      */
-    com.google.protobuf.ByteString getQualifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier();
 
     /**
      * optional uint64 timestamp = 4;
@@ -209,7 +209,7 @@ public final class CellProtos {
     /**
      * optional bytes value = 6;
      */
-    com.google.protobuf.ByteString getValue();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
 
     /**
      * optional bytes tags = 7;
@@ -218,7 +218,7 @@ public final class CellProtos {
     /**
      * optional bytes tags = 7;
      */
-    com.google.protobuf.ByteString getTags();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags();
   }
   /**
    * 
@@ -229,36 +229,36 @@ public final class CellProtos {
    * Protobuf type {@code hbase.pb.Cell}
    */
   public  static final class Cell extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Cell)
       CellOrBuilder {
     // Use Cell.newBuilder() to construct.
-    private Cell(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Cell(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Cell() {
-      row_ = com.google.protobuf.ByteString.EMPTY;
-      family_ = com.google.protobuf.ByteString.EMPTY;
-      qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       timestamp_ = 0L;
       cellType_ = 0;
-      value_ = com.google.protobuf.ByteString.EMPTY;
-      tags_ = com.google.protobuf.ByteString.EMPTY;
+      value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Cell(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -317,22 +317,22 @@ public final class CellProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_Cell_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_Cell_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -341,7 +341,7 @@ public final class CellProtos {
 
     private int bitField0_;
     public static final int ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString row_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_;
     /**
      * optional bytes row = 1;
      */
@@ -351,12 +351,12 @@ public final class CellProtos {
     /**
      * optional bytes row = 1;
      */
-    public com.google.protobuf.ByteString getRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
     public static final int FAMILY_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString family_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
     /**
      * optional bytes family = 2;
      */
@@ -366,12 +366,12 @@ public final class CellProtos {
     /**
      * optional bytes family = 2;
      */
-    public com.google.protobuf.ByteString getFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
     public static final int QUALIFIER_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString qualifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_;
     /**
      * optional bytes qualifier = 3;
      */
@@ -381,7 +381,7 @@ public final class CellProtos {
     /**
      * optional bytes qualifier = 3;
      */
-    public com.google.protobuf.ByteString getQualifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
       return qualifier_;
     }
 
@@ -417,7 +417,7 @@ public final class CellProtos {
     }
 
     public static final int VALUE_FIELD_NUMBER = 6;
-    private com.google.protobuf.ByteString value_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_;
     /**
      * optional bytes value = 6;
      */
@@ -427,12 +427,12 @@ public final class CellProtos {
     /**
      * optional bytes value = 6;
      */
-    public com.google.protobuf.ByteString getValue() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
     public static final int TAGS_FIELD_NUMBER = 7;
-    private com.google.protobuf.ByteString tags_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tags_;
     /**
      * optional bytes tags = 7;
      */
@@ -442,7 +442,7 @@ public final class CellProtos {
     /**
      * optional bytes tags = 7;
      */
-    public com.google.protobuf.ByteString getTags() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags() {
       return tags_;
     }
 
@@ -456,7 +456,7 @@ public final class CellProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, row_);
@@ -488,31 +488,31 @@ public final class CellProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, row_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, family_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, qualifier_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, timestamp_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(5, cellType_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, value_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(7, tags_);
       }
       size += unknownFields.getSerializedSize();
@@ -591,7 +591,7 @@ public final class CellProtos {
       }
       if (hasTimestamp()) {
         hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTimestamp());
       }
       if (hasCellType()) {
@@ -612,61 +612,61 @@ public final class CellProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -684,7 +684,7 @@ public final class CellProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -697,15 +697,15 @@ public final class CellProtos {
      * Protobuf type {@code hbase.pb.Cell}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Cell)
         org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.CellOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_Cell_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_Cell_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -718,35 +718,35 @@ public final class CellProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         timestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
         cellType_ = 0;
         bitField0_ = (bitField0_ & ~0x00000010);
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
-        tags_ = com.google.protobuf.ByteString.EMPTY;
+        tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_Cell_descriptor;
       }
@@ -804,29 +804,29 @@ public final class CellProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell)other);
         } else {
@@ -868,13 +868,13 @@ public final class CellProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -886,7 +886,7 @@ public final class CellProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes row = 1;
        */
@@ -896,13 +896,13 @@ public final class CellProtos {
       /**
        * optional bytes row = 1;
        */
-      public com.google.protobuf.ByteString getRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
        * optional bytes row = 1;
        */
-      public Builder setRow(com.google.protobuf.ByteString value) {
+      public Builder setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -921,7 +921,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes family = 2;
        */
@@ -931,13 +931,13 @@ public final class CellProtos {
       /**
        * optional bytes family = 2;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
        * optional bytes family = 2;
        */
-      public Builder setFamily(com.google.protobuf.ByteString value) {
+      public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -956,7 +956,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes qualifier = 3;
        */
@@ -966,13 +966,13 @@ public final class CellProtos {
       /**
        * optional bytes qualifier = 3;
        */
-      public com.google.protobuf.ByteString getQualifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
         return qualifier_;
       }
       /**
        * optional bytes qualifier = 3;
        */
-      public Builder setQualifier(com.google.protobuf.ByteString value) {
+      public Builder setQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1059,7 +1059,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes value = 6;
        */
@@ -1069,13 +1069,13 @@ public final class CellProtos {
       /**
        * optional bytes value = 6;
        */
-      public com.google.protobuf.ByteString getValue() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
         return value_;
       }
       /**
        * optional bytes value = 6;
        */
-      public Builder setValue(com.google.protobuf.ByteString value) {
+      public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1094,7 +1094,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString tags_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes tags = 7;
        */
@@ -1104,13 +1104,13 @@ public final class CellProtos {
       /**
        * optional bytes tags = 7;
        */
-      public com.google.protobuf.ByteString getTags() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags() {
         return tags_;
       }
       /**
        * optional bytes tags = 7;
        */
-      public Builder setTags(com.google.protobuf.ByteString value) {
+      public Builder setTags(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1129,12 +1129,12 @@ public final class CellProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1152,22 +1152,22 @@ public final class CellProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Cell parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Cell(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1179,7 +1179,7 @@ public final class CellProtos {
 
   public interface KeyValueOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.KeyValue)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes row = 1;
@@ -1188,7 +1188,7 @@ public final class CellProtos {
     /**
      * required bytes row = 1;
      */
-    com.google.protobuf.ByteString getRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow();
 
     /**
      * required bytes family = 2;
@@ -1197,7 +1197,7 @@ public final class CellProtos {
     /**
      * required bytes family = 2;
      */
-    com.google.protobuf.ByteString getFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
     /**
      * required bytes qualifier = 3;
@@ -1206,7 +1206,7 @@ public final class CellProtos {
     /**
      * required bytes qualifier = 3;
      */
-    com.google.protobuf.ByteString getQualifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier();
 
     /**
      * optional uint64 timestamp = 4;
@@ -1233,7 +1233,7 @@ public final class CellProtos {
     /**
      * optional bytes value = 6;
      */
-    com.google.protobuf.ByteString getValue();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
 
     /**
      * optional bytes tags = 7;
@@ -1242,7 +1242,7 @@ public final class CellProtos {
     /**
      * optional bytes tags = 7;
      */
-    com.google.protobuf.ByteString getTags();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags();
   }
   /**
    * 
@@ -1254,36 +1254,36 @@ public final class CellProtos {
    * Protobuf type {@code hbase.pb.KeyValue}
    */
   public  static final class KeyValue extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.KeyValue)
       KeyValueOrBuilder {
     // Use KeyValue.newBuilder() to construct.
-    private KeyValue(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private KeyValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private KeyValue() {
-      row_ = com.google.protobuf.ByteString.EMPTY;
-      family_ = com.google.protobuf.ByteString.EMPTY;
-      qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       timestamp_ = 0L;
       keyType_ = 0;
-      value_ = com.google.protobuf.ByteString.EMPTY;
-      tags_ = com.google.protobuf.ByteString.EMPTY;
+      value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private KeyValue(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1342,22 +1342,22 @@ public final class CellProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_KeyValue_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_KeyValue_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1366,7 +1366,7 @@ public final class CellProtos {
 
     private int bitField0_;
     public static final int ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString row_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_;
     /**
      * required bytes row = 1;
      */
@@ -1376,12 +1376,12 @@ public final class CellProtos {
     /**
      * required bytes row = 1;
      */
-    public com.google.protobuf.ByteString getRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
     public static final int FAMILY_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString family_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
     /**
      * required bytes family = 2;
      */
@@ -1391,12 +1391,12 @@ public final class CellProtos {
     /**
      * required bytes family = 2;
      */
-    public com.google.protobuf.ByteString getFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
     public static final int QUALIFIER_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString qualifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_;
     /**
      * required bytes qualifier = 3;
      */
@@ -1406,7 +1406,7 @@ public final class CellProtos {
     /**
      * required bytes qualifier = 3;
      */
-    public com.google.protobuf.ByteString getQualifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
       return qualifier_;
     }
 
@@ -1442,7 +1442,7 @@ public final class CellProtos {
     }
 
     public static final int VALUE_FIELD_NUMBER = 6;
-    private com.google.protobuf.ByteString value_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_;
     /**
      * optional bytes value = 6;
      */
@@ -1452,12 +1452,12 @@ public final class CellProtos {
     /**
      * optional bytes value = 6;
      */
-    public com.google.protobuf.ByteString getValue() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
     public static final int TAGS_FIELD_NUMBER = 7;
-    private com.google.protobuf.ByteString tags_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tags_;
     /**
      * optional bytes tags = 7;
      */
@@ -1467,7 +1467,7 @@ public final class CellProtos {
     /**
      * optional bytes tags = 7;
      */
-    public com.google.protobuf.ByteString getTags() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags() {
       return tags_;
     }
 
@@ -1493,7 +1493,7 @@ public final class CellProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, row_);
@@ -1525,31 +1525,31 @@ public final class CellProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, row_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, family_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, qualifier_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, timestamp_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(5, keyType_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, value_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(7, tags_);
       }
       size += unknownFields.getSerializedSize();
@@ -1628,7 +1628,7 @@ public final class CellProtos {
       }
       if (hasTimestamp()) {
         hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTimestamp());
       }
       if (hasKeyType()) {
@@ -1649,61 +1649,61 @@ public final class CellProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1721,7 +1721,7 @@ public final class CellProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1735,15 +1735,15 @@ public final class CellProtos {
      * Protobuf type {@code hbase.pb.KeyValue}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.KeyValue)
         org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValueOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_KeyValue_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_KeyValue_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1756,35 +1756,35 @@ public final class CellProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         timestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
         keyType_ = 0;
         bitField0_ = (bitField0_ & ~0x00000010);
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
-        tags_ = com.google.protobuf.ByteString.EMPTY;
+        tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.internal_static_hbase_pb_KeyValue_descriptor;
       }
@@ -1842,29 +1842,29 @@ public final class CellProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue)other);
         } else {
@@ -1915,13 +1915,13 @@ public final class CellProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.KeyValue) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1933,7 +1933,7 @@ public final class CellProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes row = 1;
        */
@@ -1943,13 +1943,13 @@ public final class CellProtos {
       /**
        * required bytes row = 1;
        */
-      public com.google.protobuf.ByteString getRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
        * required bytes row = 1;
        */
-      public Builder setRow(com.google.protobuf.ByteString value) {
+      public Builder setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1968,7 +1968,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family = 2;
        */
@@ -1978,13 +1978,13 @@ public final class CellProtos {
       /**
        * required bytes family = 2;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
        * required bytes family = 2;
        */
-      public Builder setFamily(com.google.protobuf.ByteString value) {
+      public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2003,7 +2003,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes qualifier = 3;
        */
@@ -2013,13 +2013,13 @@ public final class CellProtos {
       /**
        * required bytes qualifier = 3;
        */
-      public com.google.protobuf.ByteString getQualifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
         return qualifier_;
       }
       /**
        * required bytes qualifier = 3;
        */
-      public Builder setQualifier(com.google.protobuf.ByteString value) {
+      public Builder setQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2106,7 +2106,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes value = 6;
        */
@@ -2116,13 +2116,13 @@ public final class CellProtos {
       /**
        * optional bytes value = 6;
        */
-      public com.google.protobuf.ByteString getValue() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
         return value_;
       }
       /**
        * optional bytes value = 6;
        */
-      public Builder setValue(com.google.protobuf.ByteString value) {
+      public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2141,7 +2141,7 @@ public final class CellProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString tags_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes tags = 7;
        */
@@ -2151,13 +2151,13 @@ public final class CellProtos {
       /**
        * optional bytes tags = 7;
        */
-      public com.google.protobuf.ByteString getTags() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags() {
         return tags_;
       }
       /**
        * optional bytes tags = 7;
        */
-      public Builder setTags(com.google.protobuf.ByteString value) {
+      public Builder setTags(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2176,12 +2176,12 @@ public final class CellProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2199,22 +2199,22 @@ public final class CellProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public KeyValue parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new KeyValue(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2224,22 +2224,22 @@ public final class CellProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Cell_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Cell_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_KeyValue_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_KeyValue_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -2257,28 +2257,28 @@ public final class CellProtos {
       "haded.protobuf.generatedB\nCellProtosH\001\240\001" +
       "\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_Cell_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_Cell_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Cell_descriptor,
         new java.lang.String[] { "Row", "Family", "Qualifier", "Timestamp", "CellType", "Value", "Tags", });
     internal_static_hbase_pb_KeyValue_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_KeyValue_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_KeyValue_descriptor,
         new java.lang.String[] { "Row", "Family", "Qualifier", "Timestamp", "KeyType", "Value", "Tags", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
index c4ec758..80b858f 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
@@ -6,13 +6,13 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ClientProtos {
   private ClientProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * 
@@ -23,7 +23,7 @@ public final class ClientProtos {
    * Protobuf enum {@code hbase.pb.Consistency}
    */
   public enum Consistency
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * STRONG = 0;
      */
@@ -64,27 +64,27 @@ public final class ClientProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         Consistency> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public Consistency findValueByNumber(int number) {
               return Consistency.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -92,7 +92,7 @@ public final class ClientProtos {
     private static final Consistency[] VALUES = values();
 
     public static Consistency valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -111,7 +111,7 @@ public final class ClientProtos {
 
   public interface AuthorizationsOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Authorizations)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated string label = 1;
@@ -129,7 +129,7 @@ public final class ClientProtos {
     /**
      * repeated string label = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getLabelBytes(int index);
   }
   /**
@@ -141,30 +141,30 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Authorizations}
    */
   public  static final class Authorizations extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Authorizations)
       AuthorizationsOrBuilder {
     // Use Authorizations.newBuilder() to construct.
-    private Authorizations(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Authorizations(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Authorizations() {
-      label_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      label_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Authorizations(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -181,9 +181,9 @@ public final class ClientProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                label_ = new com.google.protobuf.LazyStringArrayList();
+                label_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                 mutable_bitField0_ |= 0x00000001;
               }
               label_.add(bs);
@@ -191,10 +191,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -204,12 +204,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Authorizations_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Authorizations_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -217,11 +217,11 @@ public final class ClientProtos {
     }
 
     public static final int LABEL_FIELD_NUMBER = 1;
-    private com.google.protobuf.LazyStringList label_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList label_;
     /**
      * repeated string label = 1;
      */
-    public com.google.protobuf.ProtocolStringList
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
         getLabelList() {
       return label_;
     }
@@ -240,7 +240,7 @@ public final class ClientProtos {
     /**
      * repeated string label = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getLabelBytes(int index) {
       return label_.getByteString(index);
     }
@@ -255,10 +255,10 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < label_.size(); i++) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, label_.getRaw(i));
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, label_.getRaw(i));
       }
       unknownFields.writeTo(output);
     }
@@ -316,61 +316,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -388,7 +388,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -401,15 +401,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Authorizations}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Authorizations)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.AuthorizationsOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Authorizations_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Authorizations_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -422,23 +422,23 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        label_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        label_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Authorizations_descriptor;
       }
@@ -471,29 +471,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations)other);
         } else {
@@ -524,13 +524,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Authorizations) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -542,17 +542,17 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.LazyStringList label_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList label_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureLabelIsMutable() {
         if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          label_ = new com.google.protobuf.LazyStringArrayList(label_);
+          label_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(label_);
           bitField0_ |= 0x00000001;
          }
       }
       /**
        * repeated string label = 1;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getLabelList() {
         return label_.getUnmodifiableView();
       }
@@ -571,7 +571,7 @@ public final class ClientProtos {
       /**
        * repeated string label = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getLabelBytes(int index) {
         return label_.getByteString(index);
       }
@@ -607,7 +607,7 @@ public final class ClientProtos {
       public Builder addAllLabel(
           java.lang.Iterable values) {
         ensureLabelIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, label_);
         onChanged();
         return this;
@@ -616,7 +616,7 @@ public final class ClientProtos {
        * repeated string label = 1;
        */
       public Builder clearLabel() {
-        label_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        label_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         onChanged();
         return this;
@@ -625,7 +625,7 @@ public final class ClientProtos {
        * repeated string label = 1;
        */
       public Builder addLabelBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -635,12 +635,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -658,22 +658,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Authorizations parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Authorizations(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -685,7 +685,7 @@ public final class ClientProtos {
 
   public interface CellVisibilityOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CellVisibility)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string expression = 1;
@@ -698,7 +698,7 @@ public final class ClientProtos {
     /**
      * required string expression = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getExpressionBytes();
   }
   /**
@@ -710,11 +710,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.CellVisibility}
    */
   public  static final class CellVisibility extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CellVisibility)
       CellVisibilityOrBuilder {
     // Use CellVisibility.newBuilder() to construct.
-    private CellVisibility(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CellVisibility(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CellVisibility() {
@@ -722,18 +722,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CellVisibility(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -750,29 +750,29 @@ public final class ClientProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               expression_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CellVisibility_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CellVisibility_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -796,8 +796,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           expression_ = s;
@@ -808,17 +808,17 @@ public final class ClientProtos {
     /**
      * required string expression = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getExpressionBytes() {
       java.lang.Object ref = expression_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         expression_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -836,10 +836,10 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, expression_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, expression_);
       }
       unknownFields.writeTo(output);
     }
@@ -850,7 +850,7 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, expression_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, expression_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -895,61 +895,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -967,7 +967,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -980,15 +980,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CellVisibility}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CellVisibility)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibilityOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CellVisibility_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CellVisibility_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1001,12 +1001,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -1017,7 +1017,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CellVisibility_descriptor;
       }
@@ -1051,29 +1051,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility)other);
         } else {
@@ -1102,13 +1102,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CellVisibility) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1133,8 +1133,8 @@ public final class ClientProtos {
       public java.lang.String getExpression() {
         java.lang.Object ref = expression_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             expression_ = s;
@@ -1147,17 +1147,17 @@ public final class ClientProtos {
       /**
        * required string expression = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getExpressionBytes() {
         java.lang.Object ref = expression_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           expression_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1186,7 +1186,7 @@ public final class ClientProtos {
        * required string expression = 1;
        */
       public Builder setExpressionBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1196,12 +1196,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1219,22 +1219,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CellVisibility parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CellVisibility(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1246,7 +1246,7 @@ public final class ClientProtos {
 
   public interface ColumnOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Column)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes family = 1;
@@ -1255,12 +1255,12 @@ public final class ClientProtos {
     /**
      * required bytes family = 1;
      */
-    com.google.protobuf.ByteString getFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
     /**
      * repeated bytes qualifier = 2;
      */
-    java.util.List getQualifierList();
+    java.util.List getQualifierList();
     /**
      * repeated bytes qualifier = 2;
      */
@@ -1268,7 +1268,7 @@ public final class ClientProtos {
     /**
      * repeated bytes qualifier = 2;
      */
-    com.google.protobuf.ByteString getQualifier(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier(int index);
   }
   /**
    * 
@@ -1279,31 +1279,31 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Column}
    */
   public  static final class Column extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Column)
       ColumnOrBuilder {
     // Use Column.newBuilder() to construct.
-    private Column(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Column(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Column() {
-      family_ = com.google.protobuf.ByteString.EMPTY;
+      family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       qualifier_ = java.util.Collections.emptyList();
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Column(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1326,7 +1326,7 @@ public final class ClientProtos {
             }
             case 18: {
               if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                qualifier_ = new java.util.ArrayList();
+                qualifier_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000002;
               }
               qualifier_.add(input.readBytes());
@@ -1334,10 +1334,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -1347,12 +1347,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Column_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Column_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1361,7 +1361,7 @@ public final class ClientProtos {
 
     private int bitField0_;
     public static final int FAMILY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString family_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
     /**
      * required bytes family = 1;
      */
@@ -1371,16 +1371,16 @@ public final class ClientProtos {
     /**
      * required bytes family = 1;
      */
-    public com.google.protobuf.ByteString getFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
     public static final int QUALIFIER_FIELD_NUMBER = 2;
-    private java.util.List qualifier_;
+    private java.util.List qualifier_;
     /**
      * repeated bytes qualifier = 2;
      */
-    public java.util.List
+    public java.util.List
         getQualifierList() {
       return qualifier_;
     }
@@ -1393,7 +1393,7 @@ public final class ClientProtos {
     /**
      * repeated bytes qualifier = 2;
      */
-    public com.google.protobuf.ByteString getQualifier(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier(int index) {
       return qualifier_.get(index);
     }
 
@@ -1411,7 +1411,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, family_);
@@ -1428,13 +1428,13 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, family_);
       }
       {
         int dataSize = 0;
         for (int i = 0; i < qualifier_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(qualifier_.get(i));
         }
         size += dataSize;
@@ -1489,61 +1489,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1561,7 +1561,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1574,15 +1574,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Column}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Column)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Column_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Column_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1595,25 +1595,25 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         qualifier_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Column_descriptor;
       }
@@ -1652,29 +1652,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column)other);
         } else {
@@ -1711,13 +1711,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1729,7 +1729,7 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family = 1;
        */
@@ -1739,13 +1739,13 @@ public final class ClientProtos {
       /**
        * required bytes family = 1;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
        * required bytes family = 1;
        */
-      public Builder setFamily(com.google.protobuf.ByteString value) {
+      public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1764,17 +1764,17 @@ public final class ClientProtos {
         return this;
       }
 
-      private java.util.List qualifier_ = java.util.Collections.emptyList();
+      private java.util.List qualifier_ = java.util.Collections.emptyList();
       private void ensureQualifierIsMutable() {
         if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          qualifier_ = new java.util.ArrayList(qualifier_);
+          qualifier_ = new java.util.ArrayList(qualifier_);
           bitField0_ |= 0x00000002;
          }
       }
       /**
        * repeated bytes qualifier = 2;
        */
-      public java.util.List
+      public java.util.List
           getQualifierList() {
         return java.util.Collections.unmodifiableList(qualifier_);
       }
@@ -1787,14 +1787,14 @@ public final class ClientProtos {
       /**
        * repeated bytes qualifier = 2;
        */
-      public com.google.protobuf.ByteString getQualifier(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier(int index) {
         return qualifier_.get(index);
       }
       /**
        * repeated bytes qualifier = 2;
        */
       public Builder setQualifier(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1806,7 +1806,7 @@ public final class ClientProtos {
       /**
        * repeated bytes qualifier = 2;
        */
-      public Builder addQualifier(com.google.protobuf.ByteString value) {
+      public Builder addQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1819,9 +1819,9 @@ public final class ClientProtos {
        * repeated bytes qualifier = 2;
        */
       public Builder addAllQualifier(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureQualifierIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, qualifier_);
         onChanged();
         return this;
@@ -1836,12 +1836,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1859,22 +1859,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Column parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Column(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1886,7 +1886,7 @@ public final class ClientProtos {
 
   public interface GetOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Get)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes row = 1;
@@ -1895,7 +1895,7 @@ public final class ClientProtos {
     /**
      * required bytes row = 1;
      */
-    com.google.protobuf.ByteString getRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow();
 
     /**
      * repeated .hbase.pb.Column column = 2;
@@ -2070,15 +2070,15 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Get}
    */
   public  static final class Get extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Get)
       GetOrBuilder {
     // Use Get.newBuilder() to construct.
-    private Get(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Get(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Get() {
-      row_ = com.google.protobuf.ByteString.EMPTY;
+      row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       column_ = java.util.Collections.emptyList();
       attribute_ = java.util.Collections.emptyList();
       maxVersions_ = 1;
@@ -2091,18 +2091,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Get(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2214,10 +2214,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -2233,12 +2233,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Get_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Get_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2247,7 +2247,7 @@ public final class ClientProtos {
 
     private int bitField0_;
     public static final int ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString row_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_;
     /**
      * required bytes row = 1;
      */
@@ -2257,7 +2257,7 @@ public final class ClientProtos {
     /**
      * required bytes row = 1;
      */
-    public com.google.protobuf.ByteString getRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
@@ -2547,7 +2547,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, row_);
@@ -2594,51 +2594,51 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, row_);
       }
       for (int i = 0; i < column_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, column_.get(i));
       }
       for (int i = 0; i < attribute_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, attribute_.get(i));
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getFilter());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getTimeRange());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(6, maxVersions_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(7, cacheBlocks_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(8, storeLimit_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(9, storeOffset_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(10, existenceOnly_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(12, consistency_);
       }
       for (int i = 0; i < cfTimeRange_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(13, cfTimeRange_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -2745,7 +2745,7 @@ public final class ClientProtos {
       }
       if (hasCacheBlocks()) {
         hash = (37 * hash) + CACHE_BLOCKS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCacheBlocks());
       }
       if (hasStoreLimit()) {
@@ -2758,7 +2758,7 @@ public final class ClientProtos {
       }
       if (hasExistenceOnly()) {
         hash = (37 * hash) + EXISTENCE_ONLY_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getExistenceOnly());
       }
       if (hasConsistency()) {
@@ -2775,61 +2775,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2847,7 +2847,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2862,15 +2862,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Get}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Get)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Get_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Get_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2883,12 +2883,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getColumnFieldBuilder();
           getAttributeFieldBuilder();
@@ -2899,7 +2899,7 @@ public final class ClientProtos {
       }
       public Builder clear() {
         super.clear();
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (columnBuilder_ == null) {
           column_ = java.util.Collections.emptyList();
@@ -2946,7 +2946,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Get_descriptor;
       }
@@ -3047,29 +3047,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get)other);
         } else {
@@ -3102,7 +3102,7 @@ public final class ClientProtos {
               column_ = other.column_;
               bitField0_ = (bitField0_ & ~0x00000002);
               columnBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getColumnFieldBuilder() : null;
             } else {
               columnBuilder_.addAllMessages(other.column_);
@@ -3128,7 +3128,7 @@ public final class ClientProtos {
               attribute_ = other.attribute_;
               bitField0_ = (bitField0_ & ~0x00000004);
               attributeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getAttributeFieldBuilder() : null;
             } else {
               attributeBuilder_.addAllMessages(other.attribute_);
@@ -3178,7 +3178,7 @@ public final class ClientProtos {
               cfTimeRange_ = other.cfTimeRange_;
               bitField0_ = (bitField0_ & ~0x00000800);
               cfTimeRangeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getCfTimeRangeFieldBuilder() : null;
             } else {
               cfTimeRangeBuilder_.addAllMessages(other.cfTimeRange_);
@@ -3218,13 +3218,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3236,7 +3236,7 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes row = 1;
        */
@@ -3246,13 +3246,13 @@ public final class ClientProtos {
       /**
        * required bytes row = 1;
        */
-      public com.google.protobuf.ByteString getRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
        * required bytes row = 1;
        */
-      public Builder setRow(com.google.protobuf.ByteString value) {
+      public Builder setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3280,7 +3280,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder> columnBuilder_;
 
       /**
@@ -3412,7 +3412,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (columnBuilder_ == null) {
           ensureColumnIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, column_);
           onChanged();
         } else {
@@ -3496,11 +3496,11 @@ public final class ClientProtos {
            getColumnBuilderList() {
         return getColumnFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder> 
           getColumnFieldBuilder() {
         if (columnBuilder_ == null) {
-          columnBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          columnBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder>(
                   column_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -3520,7 +3520,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> attributeBuilder_;
 
       /**
@@ -3652,7 +3652,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (attributeBuilder_ == null) {
           ensureAttributeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, attribute_);
           onChanged();
         } else {
@@ -3736,11 +3736,11 @@ public final class ClientProtos {
            getAttributeBuilderList() {
         return getAttributeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getAttributeFieldBuilder() {
         if (attributeBuilder_ == null) {
-          attributeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          attributeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   attribute_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -3752,7 +3752,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter filter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> filterBuilder_;
       /**
        * optional .hbase.pb.Filter filter = 4;
@@ -3855,11 +3855,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Filter filter = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> 
           getFilterFieldBuilder() {
         if (filterBuilder_ == null) {
-          filterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          filterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder>(
                   getFilter(),
                   getParentForChildren(),
@@ -3870,7 +3870,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange timeRange_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> timeRangeBuilder_;
       /**
        * optional .hbase.pb.TimeRange time_range = 5;
@@ -3973,11 +3973,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.TimeRange time_range = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> 
           getTimeRangeFieldBuilder() {
         if (timeRangeBuilder_ == null) {
-          timeRangeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          timeRangeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder>(
                   getTimeRange(),
                   getParentForChildren(),
@@ -4212,7 +4212,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> cfTimeRangeBuilder_;
 
       /**
@@ -4344,7 +4344,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (cfTimeRangeBuilder_ == null) {
           ensureCfTimeRangeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, cfTimeRange_);
           onChanged();
         } else {
@@ -4428,11 +4428,11 @@ public final class ClientProtos {
            getCfTimeRangeBuilderList() {
         return getCfTimeRangeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> 
           getCfTimeRangeFieldBuilder() {
         if (cfTimeRangeBuilder_ == null) {
-          cfTimeRangeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          cfTimeRangeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder>(
                   cfTimeRange_,
                   ((bitField0_ & 0x00000800) == 0x00000800),
@@ -4443,12 +4443,12 @@ public final class ClientProtos {
         return cfTimeRangeBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4466,22 +4466,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Get parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Get(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4493,7 +4493,7 @@ public final class ClientProtos {
 
   public interface ResultOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Result)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -4634,11 +4634,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Result}
    */
   public  static final class Result extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Result)
       ResultOrBuilder {
     // Use Result.newBuilder() to construct.
-    private Result(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Result(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Result() {
@@ -4650,18 +4650,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Result(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4708,10 +4708,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -4721,12 +4721,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Result_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Result_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4914,7 +4914,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < cell_.size(); i++) {
         output.writeMessage(1, cell_.get(i));
@@ -4940,23 +4940,23 @@ public final class ClientProtos {
 
       size = 0;
       for (int i = 0; i < cell_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, cell_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, associatedCellCount_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, exists_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, stale_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, partial_);
       }
       size += unknownFields.getSerializedSize();
@@ -5019,17 +5019,17 @@ public final class ClientProtos {
       }
       if (hasExists()) {
         hash = (37 * hash) + EXISTS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getExists());
       }
       if (hasStale()) {
         hash = (37 * hash) + STALE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getStale());
       }
       if (hasPartial()) {
         hash = (37 * hash) + PARTIAL_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getPartial());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -5038,61 +5038,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5110,7 +5110,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5118,15 +5118,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Result}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Result)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Result_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Result_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5139,12 +5139,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getCellFieldBuilder();
         }
@@ -5168,7 +5168,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Result_descriptor;
       }
@@ -5223,29 +5223,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result)other);
         } else {
@@ -5275,7 +5275,7 @@ public final class ClientProtos {
               cell_ = other.cell_;
               bitField0_ = (bitField0_ & ~0x00000001);
               cellBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getCellFieldBuilder() : null;
             } else {
               cellBuilder_.addAllMessages(other.cell_);
@@ -5304,13 +5304,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5331,7 +5331,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell, org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.CellOrBuilder> cellBuilder_;
 
       /**
@@ -5513,7 +5513,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (cellBuilder_ == null) {
           ensureCellIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, cell_);
           onChanged();
         } else {
@@ -5637,11 +5637,11 @@ public final class ClientProtos {
            getCellBuilderList() {
         return getCellFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell, org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.CellOrBuilder> 
           getCellFieldBuilder() {
         if (cellBuilder_ == null) {
-          cellBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          cellBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell, org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.Cell.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.CellOrBuilder>(
                   cell_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -5880,12 +5880,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5903,22 +5903,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Result parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Result(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5930,7 +5930,7 @@ public final class ClientProtos {
 
   public interface GetRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -5967,29 +5967,29 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.GetRequest}
    */
   public  static final class GetRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetRequest)
       GetRequestOrBuilder {
     // Use GetRequest.newBuilder() to construct.
-    private GetRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6033,22 +6033,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6124,7 +6124,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -6141,11 +6141,11 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getGet());
       }
       size += unknownFields.getSerializedSize();
@@ -6200,61 +6200,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6272,7 +6272,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6285,15 +6285,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.GetRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6306,12 +6306,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getGetFieldBuilder();
@@ -6334,7 +6334,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetRequest_descriptor;
       }
@@ -6380,29 +6380,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest)other);
         } else {
@@ -6441,13 +6441,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6460,7 +6460,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -6563,11 +6563,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -6578,7 +6578,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get get_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder> getBuilder_;
       /**
        * required .hbase.pb.Get get = 2;
@@ -6681,11 +6681,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.Get get = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder> 
           getGetFieldBuilder() {
         if (getBuilder_ == null) {
-          getBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          getBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder>(
                   getGet(),
                   getParentForChildren(),
@@ -6695,12 +6695,12 @@ public final class ClientProtos {
         return getBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6718,22 +6718,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6745,7 +6745,7 @@ public final class ClientProtos {
 
   public interface GetResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.Result result = 1;
@@ -6764,29 +6764,29 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.GetResponse}
    */
   public  static final class GetResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetResponse)
       GetResponseOrBuilder {
     // Use GetResponse.newBuilder() to construct.
-    private GetResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6817,22 +6817,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6871,7 +6871,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getResult());
@@ -6885,7 +6885,7 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getResult());
       }
       size += unknownFields.getSerializedSize();
@@ -6931,61 +6931,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7003,7 +7003,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7011,15 +7011,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.GetResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7032,12 +7032,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getResultFieldBuilder();
         }
@@ -7053,7 +7053,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_GetResponse_descriptor;
       }
@@ -7091,29 +7091,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse)other);
         } else {
@@ -7137,13 +7137,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7156,7 +7156,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result result_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> resultBuilder_;
       /**
        * optional .hbase.pb.Result result = 1;
@@ -7259,11 +7259,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Result result = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> 
           getResultFieldBuilder() {
         if (resultBuilder_ == null) {
-          resultBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          resultBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder>(
                   getResult(),
                   getParentForChildren(),
@@ -7273,12 +7273,12 @@ public final class ClientProtos {
         return resultBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7296,22 +7296,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7323,7 +7323,7 @@ public final class ClientProtos {
 
   public interface ConditionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Condition)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes row = 1;
@@ -7332,7 +7332,7 @@ public final class ClientProtos {
     /**
      * required bytes row = 1;
      */
-    com.google.protobuf.ByteString getRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow();
 
     /**
      * required bytes family = 2;
@@ -7341,7 +7341,7 @@ public final class ClientProtos {
     /**
      * required bytes family = 2;
      */
-    com.google.protobuf.ByteString getFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
     /**
      * required bytes qualifier = 3;
@@ -7350,7 +7350,7 @@ public final class ClientProtos {
     /**
      * required bytes qualifier = 3;
      */
-    com.google.protobuf.ByteString getQualifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier();
 
     /**
      * required .hbase.pb.CompareType compare_type = 4;
@@ -7385,33 +7385,33 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Condition}
    */
   public  static final class Condition extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Condition)
       ConditionOrBuilder {
     // Use Condition.newBuilder() to construct.
-    private Condition(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Condition(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Condition() {
-      row_ = com.google.protobuf.ByteString.EMPTY;
-      family_ = com.google.protobuf.ByteString.EMPTY;
-      qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       compareType_ = 0;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Condition(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7468,22 +7468,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Condition_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Condition_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7492,7 +7492,7 @@ public final class ClientProtos {
 
     private int bitField0_;
     public static final int ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString row_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_;
     /**
      * required bytes row = 1;
      */
@@ -7502,12 +7502,12 @@ public final class ClientProtos {
     /**
      * required bytes row = 1;
      */
-    public com.google.protobuf.ByteString getRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
     public static final int FAMILY_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString family_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
     /**
      * required bytes family = 2;
      */
@@ -7517,12 +7517,12 @@ public final class ClientProtos {
     /**
      * required bytes family = 2;
      */
-    public com.google.protobuf.ByteString getFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
     public static final int QUALIFIER_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString qualifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_;
     /**
      * required bytes qualifier = 3;
      */
@@ -7532,7 +7532,7 @@ public final class ClientProtos {
     /**
      * required bytes qualifier = 3;
      */
-    public com.google.protobuf.ByteString getQualifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
       return qualifier_;
     }
 
@@ -7607,7 +7607,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, row_);
@@ -7633,23 +7633,23 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, row_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, family_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, qualifier_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(4, compareType_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getComparator());
       }
       size += unknownFields.getSerializedSize();
@@ -7730,61 +7730,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7802,7 +7802,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7817,15 +7817,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Condition}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Condition)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Condition_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Condition_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7838,23 +7838,23 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparatorFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         compareType_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
@@ -7867,7 +7867,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Condition_descriptor;
       }
@@ -7921,29 +7921,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition)other);
         } else {
@@ -7997,13 +7997,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8015,7 +8015,7 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes row = 1;
        */
@@ -8025,13 +8025,13 @@ public final class ClientProtos {
       /**
        * required bytes row = 1;
        */
-      public com.google.protobuf.ByteString getRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
        * required bytes row = 1;
        */
-      public Builder setRow(com.google.protobuf.ByteString value) {
+      public Builder setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8050,7 +8050,7 @@ public final class ClientProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family = 2;
        */
@@ -8060,13 +8060,13 @@ public final class ClientProtos {
       /**
        * required bytes family = 2;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
        * required bytes family = 2;
        */
-      public Builder setFamily(com.google.protobuf.ByteString value) {
+      public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8085,7 +8085,7 @@ public final class ClientProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes qualifier = 3;
        */
@@ -8095,13 +8095,13 @@ public final class ClientProtos {
       /**
        * required bytes qualifier = 3;
        */
-      public com.google.protobuf.ByteString getQualifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
         return qualifier_;
       }
       /**
        * required bytes qualifier = 3;
        */
-      public Builder setQualifier(com.google.protobuf.ByteString value) {
+      public Builder setQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8157,7 +8157,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator comparator_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> comparatorBuilder_;
       /**
        * required .hbase.pb.Comparator comparator = 5;
@@ -8260,11 +8260,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.Comparator comparator = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> 
           getComparatorFieldBuilder() {
         if (comparatorBuilder_ == null) {
-          comparatorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparatorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder>(
                   getComparator(),
                   getParentForChildren(),
@@ -8274,12 +8274,12 @@ public final class ClientProtos {
         return comparatorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8297,22 +8297,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Condition parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Condition(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8324,7 +8324,7 @@ public final class ClientProtos {
 
   public interface MutationProtoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MutationProto)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes row = 1;
@@ -8333,7 +8333,7 @@ public final class ClientProtos {
     /**
      * optional bytes row = 1;
      */
-    com.google.protobuf.ByteString getRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow();
 
     /**
      * optional .hbase.pb.MutationProto.MutationType mutate_type = 2;
@@ -8487,15 +8487,15 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.MutationProto}
    */
   public  static final class MutationProto extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MutationProto)
       MutationProtoOrBuilder {
     // Use MutationProto.newBuilder() to construct.
-    private MutationProto(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MutationProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MutationProto() {
-      row_ = com.google.protobuf.ByteString.EMPTY;
+      row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       mutateType_ = 0;
       columnValue_ = java.util.Collections.emptyList();
       timestamp_ = 0L;
@@ -8506,18 +8506,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MutationProto(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8608,10 +8608,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -8624,12 +8624,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8640,7 +8640,7 @@ public final class ClientProtos {
      * Protobuf enum {@code hbase.pb.MutationProto.Durability}
      */
     public enum Durability
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * USE_DEFAULT = 0;
        */
@@ -8708,27 +8708,27 @@ public final class ClientProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Durability> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Durability findValueByNumber(int number) {
                 return Durability.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(0);
       }
@@ -8736,7 +8736,7 @@ public final class ClientProtos {
       private static final Durability[] VALUES = values();
 
       public static Durability valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -8757,7 +8757,7 @@ public final class ClientProtos {
      * Protobuf enum {@code hbase.pb.MutationProto.MutationType}
      */
     public enum MutationType
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * APPEND = 0;
        */
@@ -8816,27 +8816,27 @@ public final class ClientProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           MutationType> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public MutationType findValueByNumber(int number) {
                 return MutationType.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(1);
       }
@@ -8844,7 +8844,7 @@ public final class ClientProtos {
       private static final MutationType[] VALUES = values();
 
       public static MutationType valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -8865,7 +8865,7 @@ public final class ClientProtos {
      * Protobuf enum {@code hbase.pb.MutationProto.DeleteType}
      */
     public enum DeleteType
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * DELETE_ONE_VERSION = 0;
        */
@@ -8924,27 +8924,27 @@ public final class ClientProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           DeleteType> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public DeleteType findValueByNumber(int number) {
                 return DeleteType.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.getDescriptor().getEnumTypes().get(2);
       }
@@ -8952,7 +8952,7 @@ public final class ClientProtos {
       private static final DeleteType[] VALUES = values();
 
       public static DeleteType valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -8971,7 +8971,7 @@ public final class ClientProtos {
 
     public interface ColumnValueOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.MutationProto.ColumnValue)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required bytes family = 1;
@@ -8980,7 +8980,7 @@ public final class ClientProtos {
       /**
        * required bytes family = 1;
        */
-      com.google.protobuf.ByteString getFamily();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
       /**
        * repeated .hbase.pb.MutationProto.ColumnValue.QualifierValue qualifier_value = 2;
@@ -9010,31 +9010,31 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MutationProto.ColumnValue}
      */
     public  static final class ColumnValue extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.MutationProto.ColumnValue)
         ColumnValueOrBuilder {
       // Use ColumnValue.newBuilder() to construct.
-      private ColumnValue(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private ColumnValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private ColumnValue() {
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         qualifierValue_ = java.util.Collections.emptyList();
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private ColumnValue(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -9066,10 +9066,10 @@ public final class ClientProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -9079,12 +9079,12 @@ public final class ClientProtos {
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9093,7 +9093,7 @@ public final class ClientProtos {
 
       public interface QualifierValueOrBuilder extends
           // @@protoc_insertion_point(interface_extends:hbase.pb.MutationProto.ColumnValue.QualifierValue)
-          com.google.protobuf.MessageOrBuilder {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
         /**
          * optional bytes qualifier = 1;
@@ -9102,7 +9102,7 @@ public final class ClientProtos {
         /**
          * optional bytes qualifier = 1;
          */
-        com.google.protobuf.ByteString getQualifier();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier();
 
         /**
          * optional bytes value = 2;
@@ -9111,7 +9111,7 @@ public final class ClientProtos {
         /**
          * optional bytes value = 2;
          */
-        com.google.protobuf.ByteString getValue();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
 
         /**
          * optional uint64 timestamp = 3;
@@ -9138,40 +9138,40 @@ public final class ClientProtos {
         /**
          * optional bytes tags = 5;
          */
-        com.google.protobuf.ByteString getTags();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags();
       }
       /**
        * Protobuf type {@code hbase.pb.MutationProto.ColumnValue.QualifierValue}
        */
       public  static final class QualifierValue extends
-          com.google.protobuf.GeneratedMessageV3 implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
           // @@protoc_insertion_point(message_implements:hbase.pb.MutationProto.ColumnValue.QualifierValue)
           QualifierValueOrBuilder {
         // Use QualifierValue.newBuilder() to construct.
-        private QualifierValue(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+        private QualifierValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
           super(builder);
         }
         private QualifierValue() {
-          qualifier_ = com.google.protobuf.ByteString.EMPTY;
-          value_ = com.google.protobuf.ByteString.EMPTY;
+          qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+          value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           timestamp_ = 0L;
           deleteType_ = 0;
-          tags_ = com.google.protobuf.ByteString.EMPTY;
+          tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         }
 
         @java.lang.Override
-        public final com.google.protobuf.UnknownFieldSet
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
         getUnknownFields() {
           return this.unknownFields;
         }
         private QualifierValue(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           this();
           int mutable_bitField0_ = 0;
-          com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-              com.google.protobuf.UnknownFieldSet.newBuilder();
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
           try {
             boolean done = false;
             while (!done) {
@@ -9220,22 +9220,22 @@ public final class ClientProtos {
                 }
               }
             }
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             throw e.setUnfinishedMessage(this);
           } catch (java.io.IOException e) {
-            throw new com.google.protobuf.InvalidProtocolBufferException(
+            throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
                 e).setUnfinishedMessage(this);
           } finally {
             this.unknownFields = unknownFields.build();
             makeExtensionsImmutable();
           }
         }
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -9244,7 +9244,7 @@ public final class ClientProtos {
 
         private int bitField0_;
         public static final int QUALIFIER_FIELD_NUMBER = 1;
-        private com.google.protobuf.ByteString qualifier_;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_;
         /**
          * optional bytes qualifier = 1;
          */
@@ -9254,12 +9254,12 @@ public final class ClientProtos {
         /**
          * optional bytes qualifier = 1;
          */
-        public com.google.protobuf.ByteString getQualifier() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
           return qualifier_;
         }
 
         public static final int VALUE_FIELD_NUMBER = 2;
-        private com.google.protobuf.ByteString value_;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_;
         /**
          * optional bytes value = 2;
          */
@@ -9269,7 +9269,7 @@ public final class ClientProtos {
         /**
          * optional bytes value = 2;
          */
-        public com.google.protobuf.ByteString getValue() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
           return value_;
         }
 
@@ -9305,7 +9305,7 @@ public final class ClientProtos {
         }
 
         public static final int TAGS_FIELD_NUMBER = 5;
-        private com.google.protobuf.ByteString tags_;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tags_;
         /**
          * optional bytes tags = 5;
          */
@@ -9315,7 +9315,7 @@ public final class ClientProtos {
         /**
          * optional bytes tags = 5;
          */
-        public com.google.protobuf.ByteString getTags() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags() {
           return tags_;
         }
 
@@ -9329,7 +9329,7 @@ public final class ClientProtos {
           return true;
         }
 
-        public void writeTo(com.google.protobuf.CodedOutputStream output)
+        public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                             throws java.io.IOException {
           if (((bitField0_ & 0x00000001) == 0x00000001)) {
             output.writeBytes(1, qualifier_);
@@ -9355,23 +9355,23 @@ public final class ClientProtos {
 
           size = 0;
           if (((bitField0_ & 0x00000001) == 0x00000001)) {
-            size += com.google.protobuf.CodedOutputStream
+            size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeBytesSize(1, qualifier_);
           }
           if (((bitField0_ & 0x00000002) == 0x00000002)) {
-            size += com.google.protobuf.CodedOutputStream
+            size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeBytesSize(2, value_);
           }
           if (((bitField0_ & 0x00000004) == 0x00000004)) {
-            size += com.google.protobuf.CodedOutputStream
+            size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeUInt64Size(3, timestamp_);
           }
           if (((bitField0_ & 0x00000008) == 0x00000008)) {
-            size += com.google.protobuf.CodedOutputStream
+            size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeEnumSize(4, deleteType_);
           }
           if (((bitField0_ & 0x00000010) == 0x00000010)) {
-            size += com.google.protobuf.CodedOutputStream
+            size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeBytesSize(5, tags_);
           }
           size += unknownFields.getSerializedSize();
@@ -9436,7 +9436,7 @@ public final class ClientProtos {
           }
           if (hasTimestamp()) {
             hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
-            hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+            hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
                 getTimestamp());
           }
           if (hasDeleteType()) {
@@ -9453,61 +9453,61 @@ public final class ClientProtos {
         }
 
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(
-            com.google.protobuf.ByteString data)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return PARSER.parseFrom(data);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(
-            com.google.protobuf.ByteString data,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return PARSER.parseFrom(data, extensionRegistry);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(byte[] data)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return PARSER.parseFrom(data);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(
             byte[] data,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return PARSER.parseFrom(data, extensionRegistry);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(java.io.InputStream input)
             throws java.io.IOException {
-          return com.google.protobuf.GeneratedMessageV3
+          return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
               .parseWithIOException(PARSER, input);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(
             java.io.InputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
-          return com.google.protobuf.GeneratedMessageV3
+          return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
               .parseWithIOException(PARSER, input, extensionRegistry);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseDelimitedFrom(java.io.InputStream input)
             throws java.io.IOException {
-          return com.google.protobuf.GeneratedMessageV3
+          return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
               .parseDelimitedWithIOException(PARSER, input);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseDelimitedFrom(
             java.io.InputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
-          return com.google.protobuf.GeneratedMessageV3
+          return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
               .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(
-            com.google.protobuf.CodedInputStream input)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
             throws java.io.IOException {
-          return com.google.protobuf.GeneratedMessageV3
+          return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
               .parseWithIOException(PARSER, input);
         }
         public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parseFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
-          return com.google.protobuf.GeneratedMessageV3
+          return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
               .parseWithIOException(PARSER, input, extensionRegistry);
         }
 
@@ -9525,7 +9525,7 @@ public final class ClientProtos {
 
         @java.lang.Override
         protected Builder newBuilderForType(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           Builder builder = new Builder(parent);
           return builder;
         }
@@ -9533,15 +9533,15 @@ public final class ClientProtos {
          * Protobuf type {@code hbase.pb.MutationProto.ColumnValue.QualifierValue}
          */
         public static final class Builder extends
-            com.google.protobuf.GeneratedMessageV3.Builder implements
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
             // @@protoc_insertion_point(builder_implements:hbase.pb.MutationProto.ColumnValue.QualifierValue)
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder {
-          public static final com.google.protobuf.Descriptors.Descriptor
+          public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
               getDescriptor() {
             return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_descriptor;
           }
 
-          protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
               internalGetFieldAccessorTable() {
             return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable
                 .ensureFieldAccessorsInitialized(
@@ -9554,31 +9554,31 @@ public final class ClientProtos {
           }
 
           private Builder(
-              com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
             super(parent);
             maybeForceBuilderInitialization();
           }
           private void maybeForceBuilderInitialization() {
-            if (com.google.protobuf.GeneratedMessageV3
+            if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                     .alwaysUseFieldBuilders) {
             }
           }
           public Builder clear() {
             super.clear();
-            qualifier_ = com.google.protobuf.ByteString.EMPTY;
+            qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
             bitField0_ = (bitField0_ & ~0x00000001);
-            value_ = com.google.protobuf.ByteString.EMPTY;
+            value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
             bitField0_ = (bitField0_ & ~0x00000002);
             timestamp_ = 0L;
             bitField0_ = (bitField0_ & ~0x00000004);
             deleteType_ = 0;
             bitField0_ = (bitField0_ & ~0x00000008);
-            tags_ = com.google.protobuf.ByteString.EMPTY;
+            tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
             bitField0_ = (bitField0_ & ~0x00000010);
             return this;
           }
 
-          public com.google.protobuf.Descriptors.Descriptor
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
               getDescriptorForType() {
             return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_descriptor;
           }
@@ -9628,29 +9628,29 @@ public final class ClientProtos {
             return (Builder) super.clone();
           }
           public Builder setField(
-              com.google.protobuf.Descriptors.FieldDescriptor field,
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
               Object value) {
             return (Builder) super.setField(field, value);
           }
           public Builder clearField(
-              com.google.protobuf.Descriptors.FieldDescriptor field) {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
             return (Builder) super.clearField(field);
           }
           public Builder clearOneof(
-              com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
             return (Builder) super.clearOneof(oneof);
           }
           public Builder setRepeatedField(
-              com.google.protobuf.Descriptors.FieldDescriptor field,
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
               int index, Object value) {
             return (Builder) super.setRepeatedField(field, index, value);
           }
           public Builder addRepeatedField(
-              com.google.protobuf.Descriptors.FieldDescriptor field,
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
               Object value) {
             return (Builder) super.addRepeatedField(field, value);
           }
-          public Builder mergeFrom(com.google.protobuf.Message other) {
+          public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
             if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue) {
               return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue)other);
             } else {
@@ -9686,13 +9686,13 @@ public final class ClientProtos {
           }
 
           public Builder mergeFrom(
-              com.google.protobuf.CodedInputStream input,
-              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
               throws java.io.IOException {
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue parsedMessage = null;
             try {
               parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-            } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
               parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue) e.getUnfinishedMessage();
               throw e.unwrapIOException();
             } finally {
@@ -9704,7 +9704,7 @@ public final class ClientProtos {
           }
           private int bitField0_;
 
-          private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+          private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           /**
            * optional bytes qualifier = 1;
            */
@@ -9714,13 +9714,13 @@ public final class ClientProtos {
           /**
            * optional bytes qualifier = 1;
            */
-          public com.google.protobuf.ByteString getQualifier() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
             return qualifier_;
           }
           /**
            * optional bytes qualifier = 1;
            */
-          public Builder setQualifier(com.google.protobuf.ByteString value) {
+          public Builder setQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
             if (value == null) {
     throw new NullPointerException();
   }
@@ -9739,7 +9739,7 @@ public final class ClientProtos {
             return this;
           }
 
-          private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+          private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           /**
            * optional bytes value = 2;
            */
@@ -9749,13 +9749,13 @@ public final class ClientProtos {
           /**
            * optional bytes value = 2;
            */
-          public com.google.protobuf.ByteString getValue() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
             return value_;
           }
           /**
            * optional bytes value = 2;
            */
-          public Builder setValue(com.google.protobuf.ByteString value) {
+          public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
             if (value == null) {
     throw new NullPointerException();
   }
@@ -9842,7 +9842,7 @@ public final class ClientProtos {
             return this;
           }
 
-          private com.google.protobuf.ByteString tags_ = com.google.protobuf.ByteString.EMPTY;
+          private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tags_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           /**
            * optional bytes tags = 5;
            */
@@ -9852,13 +9852,13 @@ public final class ClientProtos {
           /**
            * optional bytes tags = 5;
            */
-          public com.google.protobuf.ByteString getTags() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTags() {
             return tags_;
           }
           /**
            * optional bytes tags = 5;
            */
-          public Builder setTags(com.google.protobuf.ByteString value) {
+          public Builder setTags(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
             if (value == null) {
     throw new NullPointerException();
   }
@@ -9877,12 +9877,12 @@ public final class ClientProtos {
             return this;
           }
           public final Builder setUnknownFields(
-              final com.google.protobuf.UnknownFieldSet unknownFields) {
+              final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
             return super.setUnknownFields(unknownFields);
           }
 
           public final Builder mergeUnknownFields(
-              final com.google.protobuf.UnknownFieldSet unknownFields) {
+              final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
             return super.mergeUnknownFields(unknownFields);
           }
 
@@ -9900,22 +9900,22 @@ public final class ClientProtos {
           return DEFAULT_INSTANCE;
         }
 
-        @java.lang.Deprecated public static final com.google.protobuf.Parser
-            PARSER = new com.google.protobuf.AbstractParser() {
+        @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+            PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
           public QualifierValue parsePartialFrom(
-              com.google.protobuf.CodedInputStream input,
-              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-              throws com.google.protobuf.InvalidProtocolBufferException {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+              throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
               return new QualifierValue(input, extensionRegistry);
           }
         };
 
-        public static com.google.protobuf.Parser parser() {
+        public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
           return PARSER;
         }
 
         @java.lang.Override
-        public com.google.protobuf.Parser getParserForType() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
           return PARSER;
         }
 
@@ -9927,7 +9927,7 @@ public final class ClientProtos {
 
       private int bitField0_;
       public static final int FAMILY_FIELD_NUMBER = 1;
-      private com.google.protobuf.ByteString family_;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
       /**
        * required bytes family = 1;
        */
@@ -9937,7 +9937,7 @@ public final class ClientProtos {
       /**
        * required bytes family = 1;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
 
@@ -9990,7 +9990,7 @@ public final class ClientProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeBytes(1, family_);
@@ -10007,11 +10007,11 @@ public final class ClientProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSize(1, family_);
         }
         for (int i = 0; i < qualifierValue_.size(); i++) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(2, qualifierValue_.get(i));
         }
         size += unknownFields.getSerializedSize();
@@ -10063,61 +10063,61 @@ public final class ClientProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -10135,7 +10135,7 @@ public final class ClientProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -10143,15 +10143,15 @@ public final class ClientProtos {
        * Protobuf type {@code hbase.pb.MutationProto.ColumnValue}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.MutationProto.ColumnValue)
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -10164,19 +10164,19 @@ public final class ClientProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
             getQualifierValueFieldBuilder();
           }
         }
         public Builder clear() {
           super.clear();
-          family_ = com.google.protobuf.ByteString.EMPTY;
+          family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           bitField0_ = (bitField0_ & ~0x00000001);
           if (qualifierValueBuilder_ == null) {
             qualifierValue_ = java.util.Collections.emptyList();
@@ -10187,7 +10187,7 @@ public final class ClientProtos {
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_ColumnValue_descriptor;
         }
@@ -10230,29 +10230,29 @@ public final class ClientProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue)other);
           } else {
@@ -10285,7 +10285,7 @@ public final class ClientProtos {
                 qualifierValue_ = other.qualifierValue_;
                 bitField0_ = (bitField0_ & ~0x00000002);
                 qualifierValueBuilder_ = 
-                  com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                  org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                      getQualifierValueFieldBuilder() : null;
               } else {
                 qualifierValueBuilder_.addAllMessages(other.qualifierValue_);
@@ -10305,13 +10305,13 @@ public final class ClientProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -10323,7 +10323,7 @@ public final class ClientProtos {
         }
         private int bitField0_;
 
-        private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         /**
          * required bytes family = 1;
          */
@@ -10333,13 +10333,13 @@ public final class ClientProtos {
         /**
          * required bytes family = 1;
          */
-        public com.google.protobuf.ByteString getFamily() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
           return family_;
         }
         /**
          * required bytes family = 1;
          */
-        public Builder setFamily(com.google.protobuf.ByteString value) {
+        public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -10367,7 +10367,7 @@ public final class ClientProtos {
            }
         }
 
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder> qualifierValueBuilder_;
 
         /**
@@ -10499,7 +10499,7 @@ public final class ClientProtos {
             java.lang.Iterable values) {
           if (qualifierValueBuilder_ == null) {
             ensureQualifierValueIsMutable();
-            com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
                 values, qualifierValue_);
             onChanged();
           } else {
@@ -10583,11 +10583,11 @@ public final class ClientProtos {
              getQualifierValueBuilderList() {
           return getQualifierValueFieldBuilder().getBuilderList();
         }
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder> 
             getQualifierValueFieldBuilder() {
           if (qualifierValueBuilder_ == null) {
-            qualifierValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+            qualifierValueBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValueOrBuilder>(
                     qualifierValue_,
                     ((bitField0_ & 0x00000002) == 0x00000002),
@@ -10598,12 +10598,12 @@ public final class ClientProtos {
           return qualifierValueBuilder_;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -10621,22 +10621,22 @@ public final class ClientProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public ColumnValue parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new ColumnValue(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -10648,7 +10648,7 @@ public final class ClientProtos {
 
     private int bitField0_;
     public static final int ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString row_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_;
     /**
      * optional bytes row = 1;
      */
@@ -10658,7 +10658,7 @@ public final class ClientProtos {
     /**
      * optional bytes row = 1;
      */
-    public com.google.protobuf.ByteString getRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
@@ -10885,7 +10885,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, row_);
@@ -10923,39 +10923,39 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, row_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, mutateType_);
       }
       for (int i = 0; i < columnValue_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, columnValue_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, timestamp_);
       }
       for (int i = 0; i < attribute_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, attribute_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(6, durability_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(7, getTimeRange());
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(8, associatedCellCount_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(9, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -11037,7 +11037,7 @@ public final class ClientProtos {
       }
       if (hasTimestamp()) {
         hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTimestamp());
       }
       if (getAttributeCount() > 0) {
@@ -11058,7 +11058,7 @@ public final class ClientProtos {
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -11067,61 +11067,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11139,7 +11139,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11156,15 +11156,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MutationProto}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MutationProto)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11177,12 +11177,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getColumnValueFieldBuilder();
           getAttributeFieldBuilder();
@@ -11191,7 +11191,7 @@ public final class ClientProtos {
       }
       public Builder clear() {
         super.clear();
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         mutateType_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
@@ -11224,7 +11224,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutationProto_descriptor;
       }
@@ -11304,29 +11304,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto)other);
         } else {
@@ -11362,7 +11362,7 @@ public final class ClientProtos {
               columnValue_ = other.columnValue_;
               bitField0_ = (bitField0_ & ~0x00000004);
               columnValueBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getColumnValueFieldBuilder() : null;
             } else {
               columnValueBuilder_.addAllMessages(other.columnValue_);
@@ -11391,7 +11391,7 @@ public final class ClientProtos {
               attribute_ = other.attribute_;
               bitField0_ = (bitField0_ & ~0x00000010);
               attributeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getAttributeFieldBuilder() : null;
             } else {
               attributeBuilder_.addAllMessages(other.attribute_);
@@ -11430,13 +11430,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11448,7 +11448,7 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes row = 1;
        */
@@ -11458,13 +11458,13 @@ public final class ClientProtos {
       /**
        * optional bytes row = 1;
        */
-      public com.google.protobuf.ByteString getRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
        * optional bytes row = 1;
        */
-      public Builder setRow(com.google.protobuf.ByteString value) {
+      public Builder setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11528,7 +11528,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder> columnValueBuilder_;
 
       /**
@@ -11660,7 +11660,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (columnValueBuilder_ == null) {
           ensureColumnValueIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, columnValue_);
           onChanged();
         } else {
@@ -11744,11 +11744,11 @@ public final class ClientProtos {
            getColumnValueBuilderList() {
         return getColumnValueFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder> 
           getColumnValueFieldBuilder() {
         if (columnValueBuilder_ == null) {
-          columnValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          columnValueBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValueOrBuilder>(
                   columnValue_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -11800,7 +11800,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> attributeBuilder_;
 
       /**
@@ -11932,7 +11932,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (attributeBuilder_ == null) {
           ensureAttributeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, attribute_);
           onChanged();
         } else {
@@ -12016,11 +12016,11 @@ public final class ClientProtos {
            getAttributeBuilderList() {
         return getAttributeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getAttributeFieldBuilder() {
         if (attributeBuilder_ == null) {
-          attributeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          attributeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   attribute_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -12068,7 +12068,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange timeRange_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> timeRangeBuilder_;
       /**
        * 
@@ -12216,11 +12216,11 @@ public final class ClientProtos {
        *
        * optional .hbase.pb.TimeRange time_range = 7;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> 
           getTimeRangeFieldBuilder() {
         if (timeRangeBuilder_ == null) {
-          timeRangeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          timeRangeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder>(
                   getTimeRange(),
                   getParentForChildren(),
@@ -12330,12 +12330,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12353,22 +12353,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MutationProto parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MutationProto(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12380,7 +12380,7 @@ public final class ClientProtos {
 
   public interface MutateRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MutateRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -12444,11 +12444,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.MutateRequest}
    */
   public  static final class MutateRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MutateRequest)
       MutateRequestOrBuilder {
     // Use MutateRequest.newBuilder() to construct.
-    private MutateRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MutateRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MutateRequest() {
@@ -12456,18 +12456,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MutateRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12529,22 +12529,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12662,7 +12662,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -12685,19 +12685,19 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getMutation());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getCondition());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonceGroup_);
       }
       size += unknownFields.getSerializedSize();
@@ -12762,7 +12762,7 @@ public final class ClientProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -12771,61 +12771,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12843,7 +12843,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12861,15 +12861,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MutateRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MutateRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12882,12 +12882,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getMutationFieldBuilder();
@@ -12919,7 +12919,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateRequest_descriptor;
       }
@@ -12977,29 +12977,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest)other);
         } else {
@@ -13049,13 +13049,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13068,7 +13068,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -13171,11 +13171,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -13186,7 +13186,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder> mutationBuilder_;
       /**
        * required .hbase.pb.MutationProto mutation = 2;
@@ -13289,11 +13289,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.MutationProto mutation = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder> 
           getMutationFieldBuilder() {
         if (mutationBuilder_ == null) {
-          mutationBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          mutationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder>(
                   getMutation(),
                   getParentForChildren(),
@@ -13304,7 +13304,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition condition_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder> conditionBuilder_;
       /**
        * optional .hbase.pb.Condition condition = 3;
@@ -13407,11 +13407,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Condition condition = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder> 
           getConditionFieldBuilder() {
         if (conditionBuilder_ == null) {
-          conditionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          conditionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder>(
                   getCondition(),
                   getParentForChildren(),
@@ -13453,12 +13453,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -13476,22 +13476,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MutateRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MutateRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13503,7 +13503,7 @@ public final class ClientProtos {
 
   public interface MutateResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MutateResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.Result result = 1;
@@ -13539,11 +13539,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.MutateResponse}
    */
   public  static final class MutateResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MutateResponse)
       MutateResponseOrBuilder {
     // Use MutateResponse.newBuilder() to construct.
-    private MutateResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MutateResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MutateResponse() {
@@ -13551,18 +13551,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MutateResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -13598,22 +13598,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -13675,7 +13675,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getResult());
@@ -13692,11 +13692,11 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getResult());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, processed_);
       }
       size += unknownFields.getSerializedSize();
@@ -13743,7 +13743,7 @@ public final class ClientProtos {
       }
       if (hasProcessed()) {
         hash = (37 * hash) + PROCESSED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getProcessed());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -13752,61 +13752,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -13824,7 +13824,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -13832,15 +13832,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MutateResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MutateResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13853,12 +13853,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getResultFieldBuilder();
         }
@@ -13876,7 +13876,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MutateResponse_descriptor;
       }
@@ -13918,29 +13918,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse)other);
         } else {
@@ -13967,13 +13967,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13986,7 +13986,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result result_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> resultBuilder_;
       /**
        * optional .hbase.pb.Result result = 1;
@@ -14089,11 +14089,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Result result = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> 
           getResultFieldBuilder() {
         if (resultBuilder_ == null) {
-          resultBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          resultBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder>(
                   getResult(),
                   getParentForChildren(),
@@ -14151,12 +14151,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14174,22 +14174,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MutateResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MutateResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14201,7 +14201,7 @@ public final class ClientProtos {
 
   public interface ScanOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Scan)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.Column column = 1;
@@ -14258,7 +14258,7 @@ public final class ClientProtos {
     /**
      * optional bytes start_row = 3;
      */
-    com.google.protobuf.ByteString getStartRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartRow();
 
     /**
      * optional bytes stop_row = 4;
@@ -14267,7 +14267,7 @@ public final class ClientProtos {
     /**
      * optional bytes stop_row = 4;
      */
-    com.google.protobuf.ByteString getStopRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRow();
 
     /**
      * optional .hbase.pb.Filter filter = 5;
@@ -14449,18 +14449,18 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Scan}
    */
   public  static final class Scan extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Scan)
       ScanOrBuilder {
     // Use Scan.newBuilder() to construct.
-    private Scan(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Scan(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Scan() {
       column_ = java.util.Collections.emptyList();
       attribute_ = java.util.Collections.emptyList();
-      startRow_ = com.google.protobuf.ByteString.EMPTY;
-      stopRow_ = com.google.protobuf.ByteString.EMPTY;
+      startRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      stopRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       maxVersions_ = 1;
       cacheBlocks_ = true;
       batchSize_ = 0;
@@ -14477,18 +14477,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Scan(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14635,10 +14635,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -14654,12 +14654,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Scan_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Scan_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14738,7 +14738,7 @@ public final class ClientProtos {
     }
 
     public static final int START_ROW_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString startRow_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString startRow_;
     /**
      * optional bytes start_row = 3;
      */
@@ -14748,12 +14748,12 @@ public final class ClientProtos {
     /**
      * optional bytes start_row = 3;
      */
-    public com.google.protobuf.ByteString getStartRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartRow() {
       return startRow_;
     }
 
     public static final int STOP_ROW_FIELD_NUMBER = 4;
-    private com.google.protobuf.ByteString stopRow_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stopRow_;
     /**
      * optional bytes stop_row = 4;
      */
@@ -14763,7 +14763,7 @@ public final class ClientProtos {
     /**
      * optional bytes stop_row = 4;
      */
-    public com.google.protobuf.ByteString getStopRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRow() {
       return stopRow_;
     }
 
@@ -15067,7 +15067,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < column_.size(); i++) {
         output.writeMessage(1, column_.get(i));
@@ -15135,79 +15135,79 @@ public final class ClientProtos {
 
       size = 0;
       for (int i = 0; i < column_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, column_.get(i));
       }
       for (int i = 0; i < attribute_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, attribute_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, startRow_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(4, stopRow_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getFilter());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, getTimeRange());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(7, maxVersions_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(8, cacheBlocks_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(9, batchSize_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(10, maxResultSize_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(11, storeLimit_);
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(12, storeOffset_);
       }
       if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(13, loadColumnFamiliesOnDemand_);
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(14, small_);
       }
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(15, reversed_);
       }
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(16, consistency_);
       }
       if (((bitField0_ & 0x00004000) == 0x00004000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(17, caching_);
       }
       if (((bitField0_ & 0x00008000) == 0x00008000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(18, allowPartialResults_);
       }
       for (int i = 0; i < cfTimeRange_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(19, cfTimeRange_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -15353,7 +15353,7 @@ public final class ClientProtos {
       }
       if (hasCacheBlocks()) {
         hash = (37 * hash) + CACHE_BLOCKS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCacheBlocks());
       }
       if (hasBatchSize()) {
@@ -15362,7 +15362,7 @@ public final class ClientProtos {
       }
       if (hasMaxResultSize()) {
         hash = (37 * hash) + MAX_RESULT_SIZE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getMaxResultSize());
       }
       if (hasStoreLimit()) {
@@ -15375,17 +15375,17 @@ public final class ClientProtos {
       }
       if (hasLoadColumnFamiliesOnDemand()) {
         hash = (37 * hash) + LOAD_COLUMN_FAMILIES_ON_DEMAND_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getLoadColumnFamiliesOnDemand());
       }
       if (hasSmall()) {
         hash = (37 * hash) + SMALL_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getSmall());
       }
       if (hasReversed()) {
         hash = (37 * hash) + REVERSED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getReversed());
       }
       if (hasConsistency()) {
@@ -15398,7 +15398,7 @@ public final class ClientProtos {
       }
       if (hasAllowPartialResults()) {
         hash = (37 * hash) + ALLOW_PARTIAL_RESULTS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getAllowPartialResults());
       }
       if (getCfTimeRangeCount() > 0) {
@@ -15411,61 +15411,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15483,7 +15483,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15501,15 +15501,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Scan}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Scan)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Scan_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Scan_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15522,12 +15522,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getColumnFieldBuilder();
           getAttributeFieldBuilder();
@@ -15550,9 +15550,9 @@ public final class ClientProtos {
         } else {
           attributeBuilder_.clear();
         }
-        startRow_ = com.google.protobuf.ByteString.EMPTY;
+        startRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
-        stopRow_ = com.google.protobuf.ByteString.EMPTY;
+        stopRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
         if (filterBuilder_ == null) {
           filter_ = null;
@@ -15599,7 +15599,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Scan_descriptor;
       }
@@ -15728,29 +15728,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan)other);
         } else {
@@ -15780,7 +15780,7 @@ public final class ClientProtos {
               column_ = other.column_;
               bitField0_ = (bitField0_ & ~0x00000001);
               columnBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getColumnFieldBuilder() : null;
             } else {
               columnBuilder_.addAllMessages(other.column_);
@@ -15806,7 +15806,7 @@ public final class ClientProtos {
               attribute_ = other.attribute_;
               bitField0_ = (bitField0_ & ~0x00000002);
               attributeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getAttributeFieldBuilder() : null;
             } else {
               attributeBuilder_.addAllMessages(other.attribute_);
@@ -15880,7 +15880,7 @@ public final class ClientProtos {
               cfTimeRange_ = other.cfTimeRange_;
               bitField0_ = (bitField0_ & ~0x00040000);
               cfTimeRangeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getCfTimeRangeFieldBuilder() : null;
             } else {
               cfTimeRangeBuilder_.addAllMessages(other.cfTimeRange_);
@@ -15917,13 +15917,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15944,7 +15944,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder> columnBuilder_;
 
       /**
@@ -16076,7 +16076,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (columnBuilder_ == null) {
           ensureColumnIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, column_);
           onChanged();
         } else {
@@ -16160,11 +16160,11 @@ public final class ClientProtos {
            getColumnBuilderList() {
         return getColumnFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder> 
           getColumnFieldBuilder() {
         if (columnBuilder_ == null) {
-          columnBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          columnBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ColumnOrBuilder>(
                   column_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -16184,7 +16184,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> attributeBuilder_;
 
       /**
@@ -16316,7 +16316,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (attributeBuilder_ == null) {
           ensureAttributeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, attribute_);
           onChanged();
         } else {
@@ -16400,11 +16400,11 @@ public final class ClientProtos {
            getAttributeBuilderList() {
         return getAttributeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getAttributeFieldBuilder() {
         if (attributeBuilder_ == null) {
-          attributeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          attributeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   attribute_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -16415,7 +16415,7 @@ public final class ClientProtos {
         return attributeBuilder_;
       }
 
-      private com.google.protobuf.ByteString startRow_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString startRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes start_row = 3;
        */
@@ -16425,13 +16425,13 @@ public final class ClientProtos {
       /**
        * optional bytes start_row = 3;
        */
-      public com.google.protobuf.ByteString getStartRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartRow() {
         return startRow_;
       }
       /**
        * optional bytes start_row = 3;
        */
-      public Builder setStartRow(com.google.protobuf.ByteString value) {
+      public Builder setStartRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -16450,7 +16450,7 @@ public final class ClientProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString stopRow_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stopRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes stop_row = 4;
        */
@@ -16460,13 +16460,13 @@ public final class ClientProtos {
       /**
        * optional bytes stop_row = 4;
        */
-      public com.google.protobuf.ByteString getStopRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRow() {
         return stopRow_;
       }
       /**
        * optional bytes stop_row = 4;
        */
-      public Builder setStopRow(com.google.protobuf.ByteString value) {
+      public Builder setStopRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -16486,7 +16486,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter filter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> filterBuilder_;
       /**
        * optional .hbase.pb.Filter filter = 5;
@@ -16589,11 +16589,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Filter filter = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> 
           getFilterFieldBuilder() {
         if (filterBuilder_ == null) {
-          filterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          filterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder>(
                   getFilter(),
                   getParentForChildren(),
@@ -16604,7 +16604,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange timeRange_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> timeRangeBuilder_;
       /**
        * optional .hbase.pb.TimeRange time_range = 6;
@@ -16707,11 +16707,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.TimeRange time_range = 6;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> 
           getTimeRangeFieldBuilder() {
         if (timeRangeBuilder_ == null) {
-          timeRangeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          timeRangeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder>(
                   getTimeRange(),
                   getParentForChildren(),
@@ -17134,7 +17134,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> cfTimeRangeBuilder_;
 
       /**
@@ -17266,7 +17266,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (cfTimeRangeBuilder_ == null) {
           ensureCfTimeRangeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, cfTimeRange_);
           onChanged();
         } else {
@@ -17350,11 +17350,11 @@ public final class ClientProtos {
            getCfTimeRangeBuilderList() {
         return getCfTimeRangeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder> 
           getCfTimeRangeFieldBuilder() {
         if (cfTimeRangeBuilder_ == null) {
-          cfTimeRangeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          cfTimeRangeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder>(
                   cfTimeRange_,
                   ((bitField0_ & 0x00040000) == 0x00040000),
@@ -17365,12 +17365,12 @@ public final class ClientProtos {
         return cfTimeRangeBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17388,22 +17388,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Scan parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Scan(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17415,7 +17415,7 @@ public final class ClientProtos {
 
   public interface ScanRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ScanRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.RegionSpecifier region = 1;
@@ -17530,11 +17530,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.ScanRequest}
    */
   public  static final class ScanRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ScanRequest)
       ScanRequestOrBuilder {
     // Use ScanRequest.newBuilder() to construct.
-    private ScanRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ScanRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ScanRequest() {
@@ -17549,18 +17549,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ScanRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -17644,22 +17644,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -17851,7 +17851,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -17892,43 +17892,43 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getScan());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, scannerId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(4, numberOfRows_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, closeScanner_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(6, nextCallSeq_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(7, clientHandlesPartials_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(8, clientHandlesHeartbeats_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(9, trackScanMetrics_);
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(10, renew_);
       }
       size += unknownFields.getSerializedSize();
@@ -18019,7 +18019,7 @@ public final class ClientProtos {
       }
       if (hasScannerId()) {
         hash = (37 * hash) + SCANNER_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getScannerId());
       }
       if (hasNumberOfRows()) {
@@ -18028,32 +18028,32 @@ public final class ClientProtos {
       }
       if (hasCloseScanner()) {
         hash = (37 * hash) + CLOSE_SCANNER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCloseScanner());
       }
       if (hasNextCallSeq()) {
         hash = (37 * hash) + NEXT_CALL_SEQ_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNextCallSeq());
       }
       if (hasClientHandlesPartials()) {
         hash = (37 * hash) + CLIENT_HANDLES_PARTIALS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getClientHandlesPartials());
       }
       if (hasClientHandlesHeartbeats()) {
         hash = (37 * hash) + CLIENT_HANDLES_HEARTBEATS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getClientHandlesHeartbeats());
       }
       if (hasTrackScanMetrics()) {
         hash = (37 * hash) + TRACK_SCAN_METRICS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getTrackScanMetrics());
       }
       if (hasRenew()) {
         hash = (37 * hash) + RENEW_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getRenew());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -18062,61 +18062,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -18134,7 +18134,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -18153,15 +18153,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.ScanRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ScanRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -18174,12 +18174,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getScanFieldBuilder();
@@ -18218,7 +18218,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanRequest_descriptor;
       }
@@ -18296,29 +18296,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest)other);
         } else {
@@ -18379,13 +18379,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -18398,7 +18398,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * optional .hbase.pb.RegionSpecifier region = 1;
@@ -18501,11 +18501,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -18516,7 +18516,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan scan_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanOrBuilder> scanBuilder_;
       /**
        * optional .hbase.pb.Scan scan = 2;
@@ -18619,11 +18619,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Scan scan = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanOrBuilder> 
           getScanFieldBuilder() {
         if (scanBuilder_ == null) {
-          scanBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          scanBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanOrBuilder>(
                   getScan(),
                   getParentForChildren(),
@@ -18889,12 +18889,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -18912,22 +18912,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ScanRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ScanRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -18939,7 +18939,7 @@ public final class ClientProtos {
 
   public interface ScanResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ScanResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -19202,11 +19202,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.ScanResponse}
    */
   public  static final class ScanResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ScanResponse)
       ScanResponseOrBuilder {
     // Use ScanResponse.newBuilder() to construct.
-    private ScanResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ScanResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ScanResponse() {
@@ -19222,18 +19222,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ScanResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19345,10 +19345,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -19364,12 +19364,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19711,7 +19711,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < cellsPerResult_.size(); i++) {
         output.writeUInt32(1, cellsPerResult_.get(i));
@@ -19754,30 +19754,30 @@ public final class ClientProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < cellsPerResult_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt32SizeNoTag(cellsPerResult_.get(i));
         }
         size += dataSize;
         size += 1 * getCellsPerResultList().size();
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, scannerId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, moreResults_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(4, ttl_);
       }
       for (int i = 0; i < results_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, results_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(6, stale_);
       }
       {
@@ -19787,15 +19787,15 @@ public final class ClientProtos {
         size += 1 * getPartialFlagPerResultList().size();
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(8, moreResultsInRegion_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(9, heartbeatMessage_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(10, getScanMetrics());
       }
       size += unknownFields.getSerializedSize();
@@ -19873,12 +19873,12 @@ public final class ClientProtos {
       }
       if (hasScannerId()) {
         hash = (37 * hash) + SCANNER_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getScannerId());
       }
       if (hasMoreResults()) {
         hash = (37 * hash) + MORE_RESULTS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getMoreResults());
       }
       if (hasTtl()) {
@@ -19891,7 +19891,7 @@ public final class ClientProtos {
       }
       if (hasStale()) {
         hash = (37 * hash) + STALE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getStale());
       }
       if (getPartialFlagPerResultCount() > 0) {
@@ -19900,12 +19900,12 @@ public final class ClientProtos {
       }
       if (hasMoreResultsInRegion()) {
         hash = (37 * hash) + MORE_RESULTS_IN_REGION_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getMoreResultsInRegion());
       }
       if (hasHeartbeatMessage()) {
         hash = (37 * hash) + HEARTBEAT_MESSAGE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getHeartbeatMessage());
       }
       if (hasScanMetrics()) {
@@ -19918,61 +19918,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -19990,7 +19990,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -20004,15 +20004,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.ScanResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ScanResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -20025,12 +20025,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getResultsFieldBuilder();
           getScanMetricsFieldBuilder();
@@ -20069,7 +20069,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ScanResponse_descriptor;
       }
@@ -20150,29 +20150,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse)other);
         } else {
@@ -20221,7 +20221,7 @@ public final class ClientProtos {
               results_ = other.results_;
               bitField0_ = (bitField0_ & ~0x00000010);
               resultsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getResultsFieldBuilder() : null;
             } else {
               resultsBuilder_.addAllMessages(other.results_);
@@ -20260,13 +20260,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20389,7 +20389,7 @@ public final class ClientProtos {
       public Builder addAllCellsPerResult(
           java.lang.Iterable values) {
         ensureCellsPerResultIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, cellsPerResult_);
         onChanged();
         return this;
@@ -20519,7 +20519,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> resultsBuilder_;
 
       /**
@@ -20711,7 +20711,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (resultsBuilder_ == null) {
           ensureResultsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, results_);
           onChanged();
         } else {
@@ -20843,11 +20843,11 @@ public final class ClientProtos {
            getResultsBuilderList() {
         return getResultsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> 
           getResultsFieldBuilder() {
         if (resultsBuilder_ == null) {
-          resultsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          resultsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder>(
                   results_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -21001,7 +21001,7 @@ public final class ClientProtos {
       public Builder addAllPartialFlagPerResult(
           java.lang.Iterable values) {
         ensurePartialFlagPerResultIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, partialFlagPerResult_);
         onChanged();
         return this;
@@ -21143,7 +21143,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics scanMetrics_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics, org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder> scanMetricsBuilder_;
       /**
        * 
@@ -21300,11 +21300,11 @@ public final class ClientProtos {
        *
        * optional .hbase.pb.ScanMetrics scan_metrics = 10;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics, org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder> 
           getScanMetricsFieldBuilder() {
         if (scanMetricsBuilder_ == null) {
-          scanMetricsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          scanMetricsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics, org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder>(
                   getScanMetrics(),
                   getParentForChildren(),
@@ -21314,12 +21314,12 @@ public final class ClientProtos {
         return scanMetricsBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -21337,22 +21337,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ScanResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ScanResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -21364,7 +21364,7 @@ public final class ClientProtos {
 
   public interface BulkLoadHFileRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BulkLoadHFileRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -21436,7 +21436,7 @@ public final class ClientProtos {
     /**
      * optional string bulk_token = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getBulkTokenBytes();
 
     /**
@@ -21458,11 +21458,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.BulkLoadHFileRequest}
    */
   public  static final class BulkLoadHFileRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BulkLoadHFileRequest)
       BulkLoadHFileRequestOrBuilder {
     // Use BulkLoadHFileRequest.newBuilder() to construct.
-    private BulkLoadHFileRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BulkLoadHFileRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BulkLoadHFileRequest() {
@@ -21473,18 +21473,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BulkLoadHFileRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -21541,7 +21541,7 @@ public final class ClientProtos {
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               bulkToken_ = bs;
               break;
@@ -21553,10 +21553,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -21566,12 +21566,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -21580,7 +21580,7 @@ public final class ClientProtos {
 
     public interface FamilyPathOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.BulkLoadHFileRequest.FamilyPath)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required bytes family = 1;
@@ -21589,7 +21589,7 @@ public final class ClientProtos {
       /**
        * required bytes family = 1;
        */
-      com.google.protobuf.ByteString getFamily();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
       /**
        * required string path = 2;
@@ -21602,38 +21602,38 @@ public final class ClientProtos {
       /**
        * required string path = 2;
        */
-      com.google.protobuf.ByteString
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getPathBytes();
     }
     /**
      * Protobuf type {@code hbase.pb.BulkLoadHFileRequest.FamilyPath}
      */
     public  static final class FamilyPath extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.BulkLoadHFileRequest.FamilyPath)
         FamilyPathOrBuilder {
       // Use FamilyPath.newBuilder() to construct.
-      private FamilyPath(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private FamilyPath(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private FamilyPath() {
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         path_ = "";
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private FamilyPath(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -21655,29 +21655,29 @@ public final class ClientProtos {
                 break;
               }
               case 18: {
-                com.google.protobuf.ByteString bs = input.readBytes();
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
                 bitField0_ |= 0x00000002;
                 path_ = bs;
                 break;
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           this.unknownFields = unknownFields.build();
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21686,7 +21686,7 @@ public final class ClientProtos {
 
       private int bitField0_;
       public static final int FAMILY_FIELD_NUMBER = 1;
-      private com.google.protobuf.ByteString family_;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
       /**
        * required bytes family = 1;
        */
@@ -21696,7 +21696,7 @@ public final class ClientProtos {
       /**
        * required bytes family = 1;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
 
@@ -21716,8 +21716,8 @@ public final class ClientProtos {
         if (ref instanceof java.lang.String) {
           return (java.lang.String) ref;
         } else {
-          com.google.protobuf.ByteString bs = 
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             path_ = s;
@@ -21728,17 +21728,17 @@ public final class ClientProtos {
       /**
        * required string path = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getPathBytes() {
         java.lang.Object ref = path_;
         if (ref instanceof java.lang.String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           path_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
 
@@ -21760,13 +21760,13 @@ public final class ClientProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeBytes(1, family_);
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          com.google.protobuf.GeneratedMessageV3.writeString(output, 2, path_);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, path_);
         }
         unknownFields.writeTo(output);
       }
@@ -21777,11 +21777,11 @@ public final class ClientProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSize(1, family_);
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, path_);
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, path_);
         }
         size += unknownFields.getSerializedSize();
         memoizedSize = size;
@@ -21835,61 +21835,61 @@ public final class ClientProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -21907,7 +21907,7 @@ public final class ClientProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -21915,15 +21915,15 @@ public final class ClientProtos {
        * Protobuf type {@code hbase.pb.BulkLoadHFileRequest.FamilyPath}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.BulkLoadHFileRequest.FamilyPath)
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPathOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -21936,25 +21936,25 @@ public final class ClientProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
           }
         }
         public Builder clear() {
           super.clear();
-          family_ = com.google.protobuf.ByteString.EMPTY;
+          family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           bitField0_ = (bitField0_ & ~0x00000001);
           path_ = "";
           bitField0_ = (bitField0_ & ~0x00000002);
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor;
         }
@@ -21992,29 +21992,29 @@ public final class ClientProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath)other);
           } else {
@@ -22049,13 +22049,13 @@ public final class ClientProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -22067,7 +22067,7 @@ public final class ClientProtos {
         }
         private int bitField0_;
 
-        private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         /**
          * required bytes family = 1;
          */
@@ -22077,13 +22077,13 @@ public final class ClientProtos {
         /**
          * required bytes family = 1;
          */
-        public com.google.protobuf.ByteString getFamily() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
           return family_;
         }
         /**
          * required bytes family = 1;
          */
-        public Builder setFamily(com.google.protobuf.ByteString value) {
+        public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -22115,8 +22115,8 @@ public final class ClientProtos {
         public java.lang.String getPath() {
           java.lang.Object ref = path_;
           if (!(ref instanceof java.lang.String)) {
-            com.google.protobuf.ByteString bs =
-                (com.google.protobuf.ByteString) ref;
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+                (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
             java.lang.String s = bs.toStringUtf8();
             if (bs.isValidUtf8()) {
               path_ = s;
@@ -22129,17 +22129,17 @@ public final class ClientProtos {
         /**
          * required string path = 2;
          */
-        public com.google.protobuf.ByteString
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
             getPathBytes() {
           java.lang.Object ref = path_;
           if (ref instanceof String) {
-            com.google.protobuf.ByteString b = 
-                com.google.protobuf.ByteString.copyFromUtf8(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                     (java.lang.String) ref);
             path_ = b;
             return b;
           } else {
-            return (com.google.protobuf.ByteString) ref;
+            return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           }
         }
         /**
@@ -22168,7 +22168,7 @@ public final class ClientProtos {
          * required string path = 2;
          */
         public Builder setPathBytes(
-            com.google.protobuf.ByteString value) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -22178,12 +22178,12 @@ public final class ClientProtos {
           return this;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -22201,22 +22201,22 @@ public final class ClientProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public FamilyPath parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new FamilyPath(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -22335,8 +22335,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           bulkToken_ = s;
@@ -22347,17 +22347,17 @@ public final class ClientProtos {
     /**
      * optional string bulk_token = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getBulkTokenBytes() {
       java.lang.Object ref = bulkToken_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         bulkToken_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -22400,7 +22400,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -22415,7 +22415,7 @@ public final class ClientProtos {
         output.writeMessage(4, getFsToken());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, bulkToken_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, bulkToken_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeBool(6, copyFile_);
@@ -22429,26 +22429,26 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       for (int i = 0; i < familyPath_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, familyPath_.get(i));
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, assignSeqNum_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getFsToken());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, bulkToken_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, bulkToken_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(6, copyFile_);
       }
       size += unknownFields.getSerializedSize();
@@ -22516,7 +22516,7 @@ public final class ClientProtos {
       }
       if (hasAssignSeqNum()) {
         hash = (37 * hash) + ASSIGN_SEQ_NUM_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getAssignSeqNum());
       }
       if (hasFsToken()) {
@@ -22529,7 +22529,7 @@ public final class ClientProtos {
       }
       if (hasCopyFile()) {
         hash = (37 * hash) + COPY_FILE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCopyFile());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -22538,61 +22538,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -22610,7 +22610,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -22624,15 +22624,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.BulkLoadHFileRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BulkLoadHFileRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -22645,12 +22645,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getFamilyPathFieldBuilder();
@@ -22686,7 +22686,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileRequest_descriptor;
       }
@@ -22753,29 +22753,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest)other);
         } else {
@@ -22808,7 +22808,7 @@ public final class ClientProtos {
               familyPath_ = other.familyPath_;
               bitField0_ = (bitField0_ & ~0x00000002);
               familyPathBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getFamilyPathFieldBuilder() : null;
             } else {
               familyPathBuilder_.addAllMessages(other.familyPath_);
@@ -22850,13 +22850,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -22869,7 +22869,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -22972,11 +22972,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -22995,7 +22995,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPathOrBuilder> familyPathBuilder_;
 
       /**
@@ -23127,7 +23127,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (familyPathBuilder_ == null) {
           ensureFamilyPathIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, familyPath_);
           onChanged();
         } else {
@@ -23211,11 +23211,11 @@ public final class ClientProtos {
            getFamilyPathBuilderList() {
         return getFamilyPathFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPathOrBuilder> 
           getFamilyPathFieldBuilder() {
         if (familyPathBuilder_ == null) {
-          familyPathBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          familyPathBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPathOrBuilder>(
                   familyPath_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -23259,7 +23259,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken fsToken_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationTokenOrBuilder> fsTokenBuilder_;
       /**
        * optional .hbase.pb.DelegationToken fs_token = 4;
@@ -23362,11 +23362,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.DelegationToken fs_token = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationTokenOrBuilder> 
           getFsTokenFieldBuilder() {
         if (fsTokenBuilder_ == null) {
-          fsTokenBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          fsTokenBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationTokenOrBuilder>(
                   getFsToken(),
                   getParentForChildren(),
@@ -23389,8 +23389,8 @@ public final class ClientProtos {
       public java.lang.String getBulkToken() {
         java.lang.Object ref = bulkToken_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             bulkToken_ = s;
@@ -23403,17 +23403,17 @@ public final class ClientProtos {
       /**
        * optional string bulk_token = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getBulkTokenBytes() {
         java.lang.Object ref = bulkToken_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           bulkToken_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -23442,7 +23442,7 @@ public final class ClientProtos {
        * optional string bulk_token = 5;
        */
       public Builder setBulkTokenBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -23484,12 +23484,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -23507,22 +23507,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BulkLoadHFileRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BulkLoadHFileRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -23534,7 +23534,7 @@ public final class ClientProtos {
 
   public interface BulkLoadHFileResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BulkLoadHFileResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool loaded = 1;
@@ -23549,11 +23549,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.BulkLoadHFileResponse}
    */
   public  static final class BulkLoadHFileResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BulkLoadHFileResponse)
       BulkLoadHFileResponseOrBuilder {
     // Use BulkLoadHFileResponse.newBuilder() to construct.
-    private BulkLoadHFileResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BulkLoadHFileResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BulkLoadHFileResponse() {
@@ -23561,18 +23561,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BulkLoadHFileResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -23595,22 +23595,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -23647,7 +23647,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, loaded_);
@@ -23661,7 +23661,7 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, loaded_);
       }
       size += unknownFields.getSerializedSize();
@@ -23699,7 +23699,7 @@ public final class ClientProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLoaded()) {
         hash = (37 * hash) + LOADED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getLoaded());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -23708,61 +23708,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -23780,7 +23780,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -23788,15 +23788,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.BulkLoadHFileResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BulkLoadHFileResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -23809,12 +23809,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -23825,7 +23825,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_BulkLoadHFileResponse_descriptor;
       }
@@ -23859,29 +23859,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse)other);
         } else {
@@ -23908,13 +23908,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -23958,12 +23958,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -23981,22 +23981,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BulkLoadHFileResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BulkLoadHFileResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -24008,7 +24008,7 @@ public final class ClientProtos {
 
   public interface DelegationTokenOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DelegationToken)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes identifier = 1;
@@ -24017,7 +24017,7 @@ public final class ClientProtos {
     /**
      * optional bytes identifier = 1;
      */
-    com.google.protobuf.ByteString getIdentifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getIdentifier();
 
     /**
      * optional bytes password = 2;
@@ -24026,7 +24026,7 @@ public final class ClientProtos {
     /**
      * optional bytes password = 2;
      */
-    com.google.protobuf.ByteString getPassword();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPassword();
 
     /**
      * optional string kind = 3;
@@ -24039,7 +24039,7 @@ public final class ClientProtos {
     /**
      * optional string kind = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getKindBytes();
 
     /**
@@ -24053,40 +24053,40 @@ public final class ClientProtos {
     /**
      * optional string service = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getServiceBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.DelegationToken}
    */
   public  static final class DelegationToken extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DelegationToken)
       DelegationTokenOrBuilder {
     // Use DelegationToken.newBuilder() to construct.
-    private DelegationToken(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DelegationToken(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DelegationToken() {
-      identifier_ = com.google.protobuf.ByteString.EMPTY;
-      password_ = com.google.protobuf.ByteString.EMPTY;
+      identifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      password_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       kind_ = "";
       service_ = "";
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DelegationToken(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -24113,35 +24113,35 @@ public final class ClientProtos {
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               kind_ = bs;
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               service_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_DelegationToken_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_DelegationToken_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -24150,7 +24150,7 @@ public final class ClientProtos {
 
     private int bitField0_;
     public static final int IDENTIFIER_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString identifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString identifier_;
     /**
      * optional bytes identifier = 1;
      */
@@ -24160,12 +24160,12 @@ public final class ClientProtos {
     /**
      * optional bytes identifier = 1;
      */
-    public com.google.protobuf.ByteString getIdentifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getIdentifier() {
       return identifier_;
     }
 
     public static final int PASSWORD_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString password_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString password_;
     /**
      * optional bytes password = 2;
      */
@@ -24175,7 +24175,7 @@ public final class ClientProtos {
     /**
      * optional bytes password = 2;
      */
-    public com.google.protobuf.ByteString getPassword() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPassword() {
       return password_;
     }
 
@@ -24195,8 +24195,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           kind_ = s;
@@ -24207,17 +24207,17 @@ public final class ClientProtos {
     /**
      * optional string kind = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getKindBytes() {
       java.lang.Object ref = kind_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         kind_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -24237,8 +24237,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           service_ = s;
@@ -24249,17 +24249,17 @@ public final class ClientProtos {
     /**
      * optional string service = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getServiceBytes() {
       java.lang.Object ref = service_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         service_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -24273,7 +24273,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, identifier_);
@@ -24282,10 +24282,10 @@ public final class ClientProtos {
         output.writeBytes(2, password_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, kind_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, kind_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, service_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, service_);
       }
       unknownFields.writeTo(output);
     }
@@ -24296,18 +24296,18 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, identifier_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, password_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, kind_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, kind_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, service_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, service_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -24379,61 +24379,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -24451,7 +24451,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -24459,15 +24459,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.DelegationToken}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DelegationToken)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationTokenOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_DelegationToken_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_DelegationToken_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -24480,20 +24480,20 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        identifier_ = com.google.protobuf.ByteString.EMPTY;
+        identifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        password_ = com.google.protobuf.ByteString.EMPTY;
+        password_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         kind_ = "";
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -24502,7 +24502,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_DelegationToken_descriptor;
       }
@@ -24548,29 +24548,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken)other);
         } else {
@@ -24607,13 +24607,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -24625,7 +24625,7 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString identifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString identifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes identifier = 1;
        */
@@ -24635,13 +24635,13 @@ public final class ClientProtos {
       /**
        * optional bytes identifier = 1;
        */
-      public com.google.protobuf.ByteString getIdentifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getIdentifier() {
         return identifier_;
       }
       /**
        * optional bytes identifier = 1;
        */
-      public Builder setIdentifier(com.google.protobuf.ByteString value) {
+      public Builder setIdentifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -24660,7 +24660,7 @@ public final class ClientProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString password_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString password_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes password = 2;
        */
@@ -24670,13 +24670,13 @@ public final class ClientProtos {
       /**
        * optional bytes password = 2;
        */
-      public com.google.protobuf.ByteString getPassword() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPassword() {
         return password_;
       }
       /**
        * optional bytes password = 2;
        */
-      public Builder setPassword(com.google.protobuf.ByteString value) {
+      public Builder setPassword(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -24708,8 +24708,8 @@ public final class ClientProtos {
       public java.lang.String getKind() {
         java.lang.Object ref = kind_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             kind_ = s;
@@ -24722,17 +24722,17 @@ public final class ClientProtos {
       /**
        * optional string kind = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getKindBytes() {
         java.lang.Object ref = kind_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           kind_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -24761,7 +24761,7 @@ public final class ClientProtos {
        * optional string kind = 3;
        */
       public Builder setKindBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -24784,8 +24784,8 @@ public final class ClientProtos {
       public java.lang.String getService() {
         java.lang.Object ref = service_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             service_ = s;
@@ -24798,17 +24798,17 @@ public final class ClientProtos {
       /**
        * optional string service = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getServiceBytes() {
         java.lang.Object ref = service_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           service_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -24837,7 +24837,7 @@ public final class ClientProtos {
        * optional string service = 4;
        */
       public Builder setServiceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -24847,12 +24847,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -24870,22 +24870,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DelegationToken parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DelegationToken(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -24897,7 +24897,7 @@ public final class ClientProtos {
 
   public interface PrepareBulkLoadRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.PrepareBulkLoadRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -24929,29 +24929,29 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.PrepareBulkLoadRequest}
    */
   public  static final class PrepareBulkLoadRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.PrepareBulkLoadRequest)
       PrepareBulkLoadRequestOrBuilder {
     // Use PrepareBulkLoadRequest.newBuilder() to construct.
-    private PrepareBulkLoadRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private PrepareBulkLoadRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private PrepareBulkLoadRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private PrepareBulkLoadRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -24995,22 +24995,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -25084,7 +25084,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -25101,11 +25101,11 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getRegion());
       }
       size += unknownFields.getSerializedSize();
@@ -25160,61 +25160,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -25232,7 +25232,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -25240,15 +25240,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.PrepareBulkLoadRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.PrepareBulkLoadRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -25261,12 +25261,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getRegionFieldBuilder();
@@ -25289,7 +25289,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadRequest_descriptor;
       }
@@ -25335,29 +25335,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest)other);
         } else {
@@ -25395,13 +25395,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -25414,7 +25414,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -25517,11 +25517,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -25532,7 +25532,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * optional .hbase.pb.RegionSpecifier region = 2;
@@ -25635,11 +25635,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.RegionSpecifier region = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -25649,12 +25649,12 @@ public final class ClientProtos {
         return regionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -25672,22 +25672,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public PrepareBulkLoadRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new PrepareBulkLoadRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -25699,7 +25699,7 @@ public final class ClientProtos {
 
   public interface PrepareBulkLoadResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.PrepareBulkLoadResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string bulk_token = 1;
@@ -25712,18 +25712,18 @@ public final class ClientProtos {
     /**
      * required string bulk_token = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getBulkTokenBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.PrepareBulkLoadResponse}
    */
   public  static final class PrepareBulkLoadResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.PrepareBulkLoadResponse)
       PrepareBulkLoadResponseOrBuilder {
     // Use PrepareBulkLoadResponse.newBuilder() to construct.
-    private PrepareBulkLoadResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private PrepareBulkLoadResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private PrepareBulkLoadResponse() {
@@ -25731,18 +25731,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private PrepareBulkLoadResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -25759,29 +25759,29 @@ public final class ClientProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               bulkToken_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -25805,8 +25805,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           bulkToken_ = s;
@@ -25817,17 +25817,17 @@ public final class ClientProtos {
     /**
      * required string bulk_token = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getBulkTokenBytes() {
       java.lang.Object ref = bulkToken_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         bulkToken_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -25845,10 +25845,10 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bulkToken_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bulkToken_);
       }
       unknownFields.writeTo(output);
     }
@@ -25859,7 +25859,7 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bulkToken_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bulkToken_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -25904,61 +25904,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -25976,7 +25976,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -25984,15 +25984,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.PrepareBulkLoadResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.PrepareBulkLoadResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -26005,12 +26005,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -26021,7 +26021,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_PrepareBulkLoadResponse_descriptor;
       }
@@ -26055,29 +26055,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse)other);
         } else {
@@ -26106,13 +26106,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -26137,8 +26137,8 @@ public final class ClientProtos {
       public java.lang.String getBulkToken() {
         java.lang.Object ref = bulkToken_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             bulkToken_ = s;
@@ -26151,17 +26151,17 @@ public final class ClientProtos {
       /**
        * required string bulk_token = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getBulkTokenBytes() {
         java.lang.Object ref = bulkToken_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           bulkToken_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -26190,7 +26190,7 @@ public final class ClientProtos {
        * required string bulk_token = 1;
        */
       public Builder setBulkTokenBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -26200,12 +26200,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -26223,22 +26223,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public PrepareBulkLoadResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new PrepareBulkLoadResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -26250,7 +26250,7 @@ public final class ClientProtos {
 
   public interface CleanupBulkLoadRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CleanupBulkLoadRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string bulk_token = 1;
@@ -26263,7 +26263,7 @@ public final class ClientProtos {
     /**
      * required string bulk_token = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getBulkTokenBytes();
 
     /**
@@ -26283,11 +26283,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.CleanupBulkLoadRequest}
    */
   public  static final class CleanupBulkLoadRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CleanupBulkLoadRequest)
       CleanupBulkLoadRequestOrBuilder {
     // Use CleanupBulkLoadRequest.newBuilder() to construct.
-    private CleanupBulkLoadRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CleanupBulkLoadRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CleanupBulkLoadRequest() {
@@ -26295,18 +26295,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CleanupBulkLoadRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -26323,7 +26323,7 @@ public final class ClientProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               bulkToken_ = bs;
               break;
@@ -26343,22 +26343,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -26382,8 +26382,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           bulkToken_ = s;
@@ -26394,17 +26394,17 @@ public final class ClientProtos {
     /**
      * required string bulk_token = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getBulkTokenBytes() {
       java.lang.Object ref = bulkToken_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         bulkToken_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -26449,10 +26449,10 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bulkToken_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bulkToken_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeMessage(2, getRegion());
@@ -26466,10 +26466,10 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bulkToken_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bulkToken_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getRegion());
       }
       size += unknownFields.getSerializedSize();
@@ -26524,61 +26524,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -26596,7 +26596,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -26604,15 +26604,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CleanupBulkLoadRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CleanupBulkLoadRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -26625,12 +26625,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -26648,7 +26648,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadRequest_descriptor;
       }
@@ -26690,29 +26690,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest)other);
         } else {
@@ -26749,13 +26749,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -26780,8 +26780,8 @@ public final class ClientProtos {
       public java.lang.String getBulkToken() {
         java.lang.Object ref = bulkToken_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             bulkToken_ = s;
@@ -26794,17 +26794,17 @@ public final class ClientProtos {
       /**
        * required string bulk_token = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getBulkTokenBytes() {
         java.lang.Object ref = bulkToken_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           bulkToken_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -26833,7 +26833,7 @@ public final class ClientProtos {
        * required string bulk_token = 1;
        */
       public Builder setBulkTokenBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -26844,7 +26844,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * optional .hbase.pb.RegionSpecifier region = 2;
@@ -26947,11 +26947,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.RegionSpecifier region = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -26961,12 +26961,12 @@ public final class ClientProtos {
         return regionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -26984,22 +26984,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CleanupBulkLoadRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CleanupBulkLoadRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -27011,34 +27011,34 @@ public final class ClientProtos {
 
   public interface CleanupBulkLoadResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CleanupBulkLoadResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.CleanupBulkLoadResponse}
    */
   public  static final class CleanupBulkLoadResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CleanupBulkLoadResponse)
       CleanupBulkLoadResponseOrBuilder {
     // Use CleanupBulkLoadResponse.newBuilder() to construct.
-    private CleanupBulkLoadResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CleanupBulkLoadResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CleanupBulkLoadResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CleanupBulkLoadResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -27056,22 +27056,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -27088,7 +27088,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -27132,61 +27132,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -27204,7 +27204,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -27212,15 +27212,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CleanupBulkLoadResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CleanupBulkLoadResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -27233,12 +27233,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -27247,7 +27247,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CleanupBulkLoadResponse_descriptor;
       }
@@ -27274,29 +27274,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse)other);
         } else {
@@ -27317,13 +27317,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -27334,12 +27334,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -27357,22 +27357,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CleanupBulkLoadResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CleanupBulkLoadResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -27384,7 +27384,7 @@ public final class ClientProtos {
 
   public interface CoprocessorServiceCallOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CoprocessorServiceCall)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes row = 1;
@@ -27393,7 +27393,7 @@ public final class ClientProtos {
     /**
      * required bytes row = 1;
      */
-    com.google.protobuf.ByteString getRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow();
 
     /**
      * required string service_name = 2;
@@ -27406,7 +27406,7 @@ public final class ClientProtos {
     /**
      * required string service_name = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getServiceNameBytes();
 
     /**
@@ -27420,7 +27420,7 @@ public final class ClientProtos {
     /**
      * required string method_name = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMethodNameBytes();
 
     /**
@@ -27430,39 +27430,39 @@ public final class ClientProtos {
     /**
      * required bytes request = 4;
      */
-    com.google.protobuf.ByteString getRequest();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRequest();
   }
   /**
    * Protobuf type {@code hbase.pb.CoprocessorServiceCall}
    */
   public  static final class CoprocessorServiceCall extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CoprocessorServiceCall)
       CoprocessorServiceCallOrBuilder {
     // Use CoprocessorServiceCall.newBuilder() to construct.
-    private CoprocessorServiceCall(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CoprocessorServiceCall(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CoprocessorServiceCall() {
-      row_ = com.google.protobuf.ByteString.EMPTY;
+      row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       serviceName_ = "";
       methodName_ = "";
-      request_ = com.google.protobuf.ByteString.EMPTY;
+      request_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CoprocessorServiceCall(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -27484,13 +27484,13 @@ public final class ClientProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               serviceName_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               methodName_ = bs;
               break;
@@ -27502,22 +27502,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceCall_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceCall_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -27526,7 +27526,7 @@ public final class ClientProtos {
 
     private int bitField0_;
     public static final int ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString row_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_;
     /**
      * required bytes row = 1;
      */
@@ -27536,7 +27536,7 @@ public final class ClientProtos {
     /**
      * required bytes row = 1;
      */
-    public com.google.protobuf.ByteString getRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
       return row_;
     }
 
@@ -27556,8 +27556,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           serviceName_ = s;
@@ -27568,17 +27568,17 @@ public final class ClientProtos {
     /**
      * required string service_name = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getServiceNameBytes() {
       java.lang.Object ref = serviceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         serviceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -27598,8 +27598,8 @@ public final class ClientProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           methodName_ = s;
@@ -27610,22 +27610,22 @@ public final class ClientProtos {
     /**
      * required string method_name = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMethodNameBytes() {
       java.lang.Object ref = methodName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         methodName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int REQUEST_FIELD_NUMBER = 4;
-    private com.google.protobuf.ByteString request_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString request_;
     /**
      * required bytes request = 4;
      */
@@ -27635,7 +27635,7 @@ public final class ClientProtos {
     /**
      * required bytes request = 4;
      */
-    public com.google.protobuf.ByteString getRequest() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRequest() {
       return request_;
     }
 
@@ -27665,16 +27665,16 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, row_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, serviceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, serviceName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, methodName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, methodName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeBytes(4, request_);
@@ -27688,17 +27688,17 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, row_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, serviceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, serviceName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, methodName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, methodName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(4, request_);
       }
       size += unknownFields.getSerializedSize();
@@ -27771,61 +27771,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -27843,7 +27843,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -27851,15 +27851,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CoprocessorServiceCall}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CoprocessorServiceCall)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceCall_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceCall_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -27872,29 +27872,29 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        row_ = com.google.protobuf.ByteString.EMPTY;
+        row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         serviceName_ = "";
         bitField0_ = (bitField0_ & ~0x00000002);
         methodName_ = "";
         bitField0_ = (bitField0_ & ~0x00000004);
-        request_ = com.google.protobuf.ByteString.EMPTY;
+        request_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceCall_descriptor;
       }
@@ -27940,29 +27940,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall)other);
         } else {
@@ -28011,13 +28011,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -28029,7 +28029,7 @@ public final class ClientProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString row_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes row = 1;
        */
@@ -28039,13 +28039,13 @@ public final class ClientProtos {
       /**
        * required bytes row = 1;
        */
-      public com.google.protobuf.ByteString getRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRow() {
         return row_;
       }
       /**
        * required bytes row = 1;
        */
-      public Builder setRow(com.google.protobuf.ByteString value) {
+      public Builder setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -28077,8 +28077,8 @@ public final class ClientProtos {
       public java.lang.String getServiceName() {
         java.lang.Object ref = serviceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             serviceName_ = s;
@@ -28091,17 +28091,17 @@ public final class ClientProtos {
       /**
        * required string service_name = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getServiceNameBytes() {
         java.lang.Object ref = serviceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           serviceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -28130,7 +28130,7 @@ public final class ClientProtos {
        * required string service_name = 2;
        */
       public Builder setServiceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -28153,8 +28153,8 @@ public final class ClientProtos {
       public java.lang.String getMethodName() {
         java.lang.Object ref = methodName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             methodName_ = s;
@@ -28167,17 +28167,17 @@ public final class ClientProtos {
       /**
        * required string method_name = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getMethodNameBytes() {
         java.lang.Object ref = methodName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           methodName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -28206,7 +28206,7 @@ public final class ClientProtos {
        * required string method_name = 3;
        */
       public Builder setMethodNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -28216,7 +28216,7 @@ public final class ClientProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString request_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString request_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes request = 4;
        */
@@ -28226,13 +28226,13 @@ public final class ClientProtos {
       /**
        * required bytes request = 4;
        */
-      public com.google.protobuf.ByteString getRequest() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRequest() {
         return request_;
       }
       /**
        * required bytes request = 4;
        */
-      public Builder setRequest(com.google.protobuf.ByteString value) {
+      public Builder setRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -28251,12 +28251,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -28274,22 +28274,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CoprocessorServiceCall parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CoprocessorServiceCall(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -28301,7 +28301,7 @@ public final class ClientProtos {
 
   public interface CoprocessorServiceResultOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CoprocessorServiceResult)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.NameBytesPair value = 1;
@@ -28320,29 +28320,29 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.CoprocessorServiceResult}
    */
   public  static final class CoprocessorServiceResult extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CoprocessorServiceResult)
       CoprocessorServiceResultOrBuilder {
     // Use CoprocessorServiceResult.newBuilder() to construct.
-    private CoprocessorServiceResult(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CoprocessorServiceResult(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CoprocessorServiceResult() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CoprocessorServiceResult(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -28373,22 +28373,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResult_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResult_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -28433,7 +28433,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getValue());
@@ -28447,7 +28447,7 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getValue());
       }
       size += unknownFields.getSerializedSize();
@@ -28493,61 +28493,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -28565,7 +28565,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -28573,15 +28573,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CoprocessorServiceResult}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CoprocessorServiceResult)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResult_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResult_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -28594,12 +28594,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getValueFieldBuilder();
         }
@@ -28615,7 +28615,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResult_descriptor;
       }
@@ -28653,29 +28653,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult)other);
         } else {
@@ -28704,13 +28704,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -28723,7 +28723,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair value_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> valueBuilder_;
       /**
        * optional .hbase.pb.NameBytesPair value = 1;
@@ -28826,11 +28826,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.NameBytesPair value = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getValueFieldBuilder() {
         if (valueBuilder_ == null) {
-          valueBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          valueBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   getValue(),
                   getParentForChildren(),
@@ -28840,12 +28840,12 @@ public final class ClientProtos {
         return valueBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -28863,22 +28863,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CoprocessorServiceResult parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CoprocessorServiceResult(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -28890,7 +28890,7 @@ public final class ClientProtos {
 
   public interface CoprocessorServiceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CoprocessorServiceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -28922,29 +28922,29 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.CoprocessorServiceRequest}
    */
   public  static final class CoprocessorServiceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CoprocessorServiceRequest)
       CoprocessorServiceRequestOrBuilder {
     // Use CoprocessorServiceRequest.newBuilder() to construct.
-    private CoprocessorServiceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CoprocessorServiceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CoprocessorServiceRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CoprocessorServiceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -28988,22 +28988,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -29079,7 +29079,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -29096,11 +29096,11 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getCall());
       }
       size += unknownFields.getSerializedSize();
@@ -29155,61 +29155,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -29227,7 +29227,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -29235,15 +29235,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CoprocessorServiceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CoprocessorServiceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -29256,12 +29256,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getCallFieldBuilder();
@@ -29284,7 +29284,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceRequest_descriptor;
       }
@@ -29330,29 +29330,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest)other);
         } else {
@@ -29391,13 +29391,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -29410,7 +29410,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -29513,11 +29513,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -29528,7 +29528,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall call_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder> callBuilder_;
       /**
        * required .hbase.pb.CoprocessorServiceCall call = 2;
@@ -29631,11 +29631,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.CoprocessorServiceCall call = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder> 
           getCallFieldBuilder() {
         if (callBuilder_ == null) {
-          callBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          callBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder>(
                   getCall(),
                   getParentForChildren(),
@@ -29645,12 +29645,12 @@ public final class ClientProtos {
         return callBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -29668,22 +29668,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CoprocessorServiceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CoprocessorServiceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -29695,7 +29695,7 @@ public final class ClientProtos {
 
   public interface CoprocessorServiceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CoprocessorServiceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -29727,29 +29727,29 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.CoprocessorServiceResponse}
    */
   public  static final class CoprocessorServiceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CoprocessorServiceResponse)
       CoprocessorServiceResponseOrBuilder {
     // Use CoprocessorServiceResponse.newBuilder() to construct.
-    private CoprocessorServiceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CoprocessorServiceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CoprocessorServiceResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CoprocessorServiceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -29793,22 +29793,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -29884,7 +29884,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -29901,11 +29901,11 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getValue());
       }
       size += unknownFields.getSerializedSize();
@@ -29960,61 +29960,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -30032,7 +30032,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -30040,15 +30040,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.CoprocessorServiceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CoprocessorServiceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -30061,12 +30061,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getValueFieldBuilder();
@@ -30089,7 +30089,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_CoprocessorServiceResponse_descriptor;
       }
@@ -30135,29 +30135,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse)other);
         } else {
@@ -30196,13 +30196,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -30215,7 +30215,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -30318,11 +30318,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -30333,7 +30333,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair value_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> valueBuilder_;
       /**
        * required .hbase.pb.NameBytesPair value = 2;
@@ -30436,11 +30436,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.NameBytesPair value = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getValueFieldBuilder() {
         if (valueBuilder_ == null) {
-          valueBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          valueBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   getValue(),
                   getParentForChildren(),
@@ -30450,12 +30450,12 @@ public final class ClientProtos {
         return valueBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -30473,22 +30473,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CoprocessorServiceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CoprocessorServiceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -30500,7 +30500,7 @@ public final class ClientProtos {
 
   public interface ActionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Action)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -30568,11 +30568,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.Action}
    */
   public  static final class Action extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Action)
       ActionOrBuilder {
     // Use Action.newBuilder() to construct.
-    private Action(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Action(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Action() {
@@ -30580,18 +30580,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Action(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -30653,22 +30653,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Action_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Action_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -30792,7 +30792,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, index_);
@@ -30815,19 +30815,19 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, index_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getMutation());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getGet());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getServiceCall());
       }
       size += unknownFields.getSerializedSize();
@@ -30900,61 +30900,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -30972,7 +30972,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -30984,15 +30984,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.Action}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Action)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ActionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Action_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Action_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -31005,12 +31005,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getMutationFieldBuilder();
           getGetFieldBuilder();
@@ -31042,7 +31042,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_Action_descriptor;
       }
@@ -31100,29 +31100,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action)other);
         } else {
@@ -31170,13 +31170,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -31241,7 +31241,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto mutation_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder> mutationBuilder_;
       /**
        * optional .hbase.pb.MutationProto mutation = 2;
@@ -31344,11 +31344,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.MutationProto mutation = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder> 
           getMutationFieldBuilder() {
         if (mutationBuilder_ == null) {
-          mutationBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          mutationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProtoOrBuilder>(
                   getMutation(),
                   getParentForChildren(),
@@ -31359,7 +31359,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get get_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder> getBuilder_;
       /**
        * optional .hbase.pb.Get get = 3;
@@ -31462,11 +31462,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Get get = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder> 
           getGetFieldBuilder() {
         if (getBuilder_ == null) {
-          getBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          getBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetOrBuilder>(
                   getGet(),
                   getParentForChildren(),
@@ -31477,7 +31477,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder> serviceCallBuilder_;
       /**
        * optional .hbase.pb.CoprocessorServiceCall service_call = 4;
@@ -31580,11 +31580,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.CoprocessorServiceCall service_call = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder> 
           getServiceCallFieldBuilder() {
         if (serviceCallBuilder_ == null) {
-          serviceCallBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serviceCallBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder>(
                   getServiceCall(),
                   getParentForChildren(),
@@ -31594,12 +31594,12 @@ public final class ClientProtos {
         return serviceCallBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -31617,22 +31617,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Action parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Action(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -31644,7 +31644,7 @@ public final class ClientProtos {
 
   public interface RegionActionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionAction)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -31709,11 +31709,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.RegionAction}
    */
   public  static final class RegionAction extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionAction)
       RegionActionOrBuilder {
     // Use RegionAction.newBuilder() to construct.
-    private RegionAction(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionAction(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionAction() {
@@ -31722,18 +31722,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionAction(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -31778,10 +31778,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -31791,12 +31791,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionAction_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionAction_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -31907,7 +31907,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -31927,15 +31927,15 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, atomic_);
       }
       for (int i = 0; i < action_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, action_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -31984,7 +31984,7 @@ public final class ClientProtos {
       }
       if (hasAtomic()) {
         hash = (37 * hash) + ATOMIC_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getAtomic());
       }
       if (getActionCount() > 0) {
@@ -31997,61 +31997,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -32069,7 +32069,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -32082,15 +32082,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.RegionAction}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionAction)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionAction_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionAction_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -32103,12 +32103,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getActionFieldBuilder();
@@ -32133,7 +32133,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionAction_descriptor;
       }
@@ -32184,29 +32184,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction)other);
         } else {
@@ -32242,7 +32242,7 @@ public final class ClientProtos {
               action_ = other.action_;
               bitField0_ = (bitField0_ & ~0x00000004);
               actionBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getActionFieldBuilder() : null;
             } else {
               actionBuilder_.addAllMessages(other.action_);
@@ -32270,13 +32270,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -32289,7 +32289,7 @@ public final class ClientProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -32392,11 +32392,11 @@ public final class ClientProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -32463,7 +32463,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ActionOrBuilder> actionBuilder_;
 
       /**
@@ -32595,7 +32595,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (actionBuilder_ == null) {
           ensureActionIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, action_);
           onChanged();
         } else {
@@ -32679,11 +32679,11 @@ public final class ClientProtos {
            getActionBuilderList() {
         return getActionFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ActionOrBuilder> 
           getActionFieldBuilder() {
         if (actionBuilder_ == null) {
-          actionBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          actionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ActionOrBuilder>(
                   action_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -32694,12 +32694,12 @@ public final class ClientProtos {
         return actionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -32717,22 +32717,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionAction parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionAction(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -32744,7 +32744,7 @@ public final class ClientProtos {
 
   public interface RegionLoadStatsOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionLoadStats)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -32807,11 +32807,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.RegionLoadStats}
    */
   public  static final class RegionLoadStats extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionLoadStats)
       RegionLoadStatsOrBuilder {
     // Use RegionLoadStats.newBuilder() to construct.
-    private RegionLoadStats(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionLoadStats(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionLoadStats() {
@@ -32821,18 +32821,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionLoadStats(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -32865,22 +32865,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionLoadStats_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionLoadStats_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -32969,7 +32969,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(1, memstoreLoad_);
@@ -32989,15 +32989,15 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(1, memstoreLoad_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, heapOccupancy_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(3, compactionPressure_);
       }
       size += unknownFields.getSerializedSize();
@@ -33061,61 +33061,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -33133,7 +33133,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -33145,15 +33145,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.RegionLoadStats}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionLoadStats)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionLoadStats_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionLoadStats_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -33166,12 +33166,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -33186,7 +33186,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionLoadStats_descriptor;
       }
@@ -33228,29 +33228,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats)other);
         } else {
@@ -33280,13 +33280,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -33446,12 +33446,12 @@ public final class ClientProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -33469,22 +33469,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionLoadStats parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionLoadStats(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -33496,7 +33496,7 @@ public final class ClientProtos {
 
   public interface MultiRegionLoadStatsOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MultiRegionLoadStats)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.RegionSpecifier region = 1;
@@ -33550,11 +33550,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.MultiRegionLoadStats}
    */
   public  static final class MultiRegionLoadStats extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MultiRegionLoadStats)
       MultiRegionLoadStatsOrBuilder {
     // Use MultiRegionLoadStats.newBuilder() to construct.
-    private MultiRegionLoadStats(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MultiRegionLoadStats(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MultiRegionLoadStats() {
@@ -33563,18 +33563,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MultiRegionLoadStats(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -33610,10 +33610,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -33626,12 +33626,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRegionLoadStats_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRegionLoadStats_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -33724,7 +33724,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < region_.size(); i++) {
         output.writeMessage(1, region_.get(i));
@@ -33741,11 +33741,11 @@ public final class ClientProtos {
 
       size = 0;
       for (int i = 0; i < region_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, region_.get(i));
       }
       for (int i = 0; i < stat_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, stat_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -33794,61 +33794,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -33866,7 +33866,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -33874,15 +33874,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MultiRegionLoadStats}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MultiRegionLoadStats)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStatsOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRegionLoadStats_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRegionLoadStats_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -33895,12 +33895,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getStatFieldBuilder();
@@ -33923,7 +33923,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRegionLoadStats_descriptor;
       }
@@ -33969,29 +33969,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats)other);
         } else {
@@ -34021,7 +34021,7 @@ public final class ClientProtos {
               region_ = other.region_;
               bitField0_ = (bitField0_ & ~0x00000001);
               regionBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionFieldBuilder() : null;
             } else {
               regionBuilder_.addAllMessages(other.region_);
@@ -34047,7 +34047,7 @@ public final class ClientProtos {
               stat_ = other.stat_;
               bitField0_ = (bitField0_ & ~0x00000002);
               statBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStatFieldBuilder() : null;
             } else {
               statBuilder_.addAllMessages(other.stat_);
@@ -34069,13 +34069,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -34096,7 +34096,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
 
       /**
@@ -34228,7 +34228,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (regionBuilder_ == null) {
           ensureRegionIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, region_);
           onChanged();
         } else {
@@ -34312,11 +34312,11 @@ public final class ClientProtos {
            getRegionBuilderList() {
         return getRegionFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   region_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -34336,7 +34336,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> statBuilder_;
 
       /**
@@ -34468,7 +34468,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (statBuilder_ == null) {
           ensureStatIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, stat_);
           onChanged();
         } else {
@@ -34552,11 +34552,11 @@ public final class ClientProtos {
            getStatBuilderList() {
         return getStatFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> 
           getStatFieldBuilder() {
         if (statBuilder_ == null) {
-          statBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          statBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder>(
                   stat_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -34567,12 +34567,12 @@ public final class ClientProtos {
         return statBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -34590,22 +34590,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MultiRegionLoadStats parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MultiRegionLoadStats(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -34617,7 +34617,7 @@ public final class ClientProtos {
 
   public interface ResultOrExceptionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ResultOrException)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -34725,11 +34725,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.ResultOrException}
    */
   public  static final class ResultOrException extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ResultOrException)
       ResultOrExceptionOrBuilder {
     // Use ResultOrException.newBuilder() to construct.
-    private ResultOrException(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ResultOrException(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ResultOrException() {
@@ -34737,18 +34737,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ResultOrException(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -34823,22 +34823,22 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ResultOrException_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ResultOrException_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -35001,7 +35001,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, index_);
@@ -35027,23 +35027,23 @@ public final class ClientProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, index_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getResult());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getException());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getServiceResult());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getLoadStats());
       }
       size += unknownFields.getSerializedSize();
@@ -35125,61 +35125,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -35197,7 +35197,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -35212,15 +35212,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.ResultOrException}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ResultOrException)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ResultOrException_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ResultOrException_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -35233,12 +35233,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getResultFieldBuilder();
           getExceptionFieldBuilder();
@@ -35277,7 +35277,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_ResultOrException_descriptor;
       }
@@ -35343,29 +35343,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException)other);
         } else {
@@ -35411,13 +35411,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -35482,7 +35482,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result result_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> resultBuilder_;
       /**
        * optional .hbase.pb.Result result = 2;
@@ -35585,11 +35585,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Result result = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder> 
           getResultFieldBuilder() {
         if (resultBuilder_ == null) {
-          resultBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          resultBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrBuilder>(
                   getResult(),
                   getParentForChildren(),
@@ -35600,7 +35600,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair exception_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> exceptionBuilder_;
       /**
        * optional .hbase.pb.NameBytesPair exception = 3;
@@ -35703,11 +35703,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.NameBytesPair exception = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getExceptionFieldBuilder() {
         if (exceptionBuilder_ == null) {
-          exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          exceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   getException(),
                   getParentForChildren(),
@@ -35718,7 +35718,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult serviceResult_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder> serviceResultBuilder_;
       /**
        * 
@@ -35857,11 +35857,11 @@ public final class ClientProtos {
        *
        * optional .hbase.pb.CoprocessorServiceResult service_result = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder> 
           getServiceResultFieldBuilder() {
         if (serviceResultBuilder_ == null) {
-          serviceResultBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serviceResultBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder>(
                   getServiceResult(),
                   getParentForChildren(),
@@ -35872,7 +35872,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats loadStats_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> loadStatsBuilder_;
       /**
        * 
@@ -36011,11 +36011,11 @@ public final class ClientProtos {
        *
        * optional .hbase.pb.RegionLoadStats loadStats = 5 [deprecated = true];
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder> 
           getLoadStatsFieldBuilder() {
         if (loadStatsBuilder_ == null) {
-          loadStatsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          loadStatsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionLoadStatsOrBuilder>(
                   getLoadStats(),
                   getParentForChildren(),
@@ -36025,12 +36025,12 @@ public final class ClientProtos {
         return loadStatsBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -36048,22 +36048,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ResultOrException parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ResultOrException(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -36075,7 +36075,7 @@ public final class ClientProtos {
 
   public interface RegionActionResultOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionActionResult)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.ResultOrException resultOrException = 1;
@@ -36135,11 +36135,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.RegionActionResult}
    */
   public  static final class RegionActionResult extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionActionResult)
       RegionActionResultOrBuilder {
     // Use RegionActionResult.newBuilder() to construct.
-    private RegionActionResult(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionActionResult(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionActionResult() {
@@ -36147,18 +36147,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionActionResult(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -36198,10 +36198,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -36211,12 +36211,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionActionResult_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionActionResult_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -36314,7 +36314,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < resultOrException_.size(); i++) {
         output.writeMessage(1, resultOrException_.get(i));
@@ -36331,11 +36331,11 @@ public final class ClientProtos {
 
       size = 0;
       for (int i = 0; i < resultOrException_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, resultOrException_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getException());
       }
       size += unknownFields.getSerializedSize();
@@ -36387,61 +36387,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -36459,7 +36459,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -36472,15 +36472,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.RegionActionResult}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionActionResult)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResultOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionActionResult_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionActionResult_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -36493,12 +36493,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getResultOrExceptionFieldBuilder();
           getExceptionFieldBuilder();
@@ -36521,7 +36521,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_RegionActionResult_descriptor;
       }
@@ -36568,29 +36568,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult)other);
         } else {
@@ -36620,7 +36620,7 @@ public final class ClientProtos {
               resultOrException_ = other.resultOrException_;
               bitField0_ = (bitField0_ & ~0x00000001);
               resultOrExceptionBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getResultOrExceptionFieldBuilder() : null;
             } else {
               resultOrExceptionBuilder_.addAllMessages(other.resultOrException_);
@@ -36650,13 +36650,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -36677,7 +36677,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder> resultOrExceptionBuilder_;
 
       /**
@@ -36809,7 +36809,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (resultOrExceptionBuilder_ == null) {
           ensureResultOrExceptionIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, resultOrException_);
           onChanged();
         } else {
@@ -36893,11 +36893,11 @@ public final class ClientProtos {
            getResultOrExceptionBuilderList() {
         return getResultOrExceptionFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder> 
           getResultOrExceptionFieldBuilder() {
         if (resultOrExceptionBuilder_ == null) {
-          resultOrExceptionBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          resultOrExceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder>(
                   resultOrException_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -36909,7 +36909,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair exception_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> exceptionBuilder_;
       /**
        * 
@@ -37048,11 +37048,11 @@ public final class ClientProtos {
        *
        * optional .hbase.pb.NameBytesPair exception = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> 
           getExceptionFieldBuilder() {
         if (exceptionBuilder_ == null) {
-          exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          exceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
                   getException(),
                   getParentForChildren(),
@@ -37062,12 +37062,12 @@ public final class ClientProtos {
         return exceptionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -37085,22 +37085,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionActionResult parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionActionResult(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -37112,7 +37112,7 @@ public final class ClientProtos {
 
   public interface MultiRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MultiRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.RegionAction regionAction = 1;
@@ -37173,11 +37173,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.MultiRequest}
    */
   public  static final class MultiRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MultiRequest)
       MultiRequestOrBuilder {
     // Use MultiRequest.newBuilder() to construct.
-    private MultiRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MultiRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MultiRequest() {
@@ -37186,18 +37186,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MultiRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -37242,10 +37242,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -37255,12 +37255,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -37361,7 +37361,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < regionAction_.size(); i++) {
         output.writeMessage(1, regionAction_.get(i));
@@ -37381,15 +37381,15 @@ public final class ClientProtos {
 
       size = 0;
       for (int i = 0; i < regionAction_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, regionAction_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getCondition());
       }
       size += unknownFields.getSerializedSize();
@@ -37438,7 +37438,7 @@ public final class ClientProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCEGROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasCondition()) {
@@ -37451,61 +37451,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -37523,7 +37523,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -37540,15 +37540,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MultiRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MultiRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -37561,12 +37561,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionActionFieldBuilder();
           getConditionFieldBuilder();
@@ -37591,7 +37591,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiRequest_descriptor;
       }
@@ -37642,29 +37642,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest)other);
         } else {
@@ -37694,7 +37694,7 @@ public final class ClientProtos {
               regionAction_ = other.regionAction_;
               bitField0_ = (bitField0_ & ~0x00000001);
               regionActionBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionActionFieldBuilder() : null;
             } else {
               regionActionBuilder_.addAllMessages(other.regionAction_);
@@ -37727,13 +37727,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -37754,7 +37754,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionOrBuilder> regionActionBuilder_;
 
       /**
@@ -37886,7 +37886,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (regionActionBuilder_ == null) {
           ensureRegionActionIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionAction_);
           onChanged();
         } else {
@@ -37970,11 +37970,11 @@ public final class ClientProtos {
            getRegionActionBuilderList() {
         return getRegionActionFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionOrBuilder> 
           getRegionActionFieldBuilder() {
         if (regionActionBuilder_ == null) {
-          regionActionBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionActionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionOrBuilder>(
                   regionAction_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -38018,7 +38018,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition condition_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder> conditionBuilder_;
       /**
        * optional .hbase.pb.Condition condition = 3;
@@ -38121,11 +38121,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.Condition condition = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder> 
           getConditionFieldBuilder() {
         if (conditionBuilder_ == null) {
-          conditionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          conditionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ConditionOrBuilder>(
                   getCondition(),
                   getParentForChildren(),
@@ -38135,12 +38135,12 @@ public final class ClientProtos {
         return conditionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -38158,22 +38158,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MultiRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MultiRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -38185,7 +38185,7 @@ public final class ClientProtos {
 
   public interface MultiResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MultiResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.RegionActionResult regionActionResult = 1;
@@ -38245,11 +38245,11 @@ public final class ClientProtos {
    * Protobuf type {@code hbase.pb.MultiResponse}
    */
   public  static final class MultiResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MultiResponse)
       MultiResponseOrBuilder {
     // Use MultiResponse.newBuilder() to construct.
-    private MultiResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MultiResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MultiResponse() {
@@ -38258,18 +38258,18 @@ public final class ClientProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MultiResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -38314,10 +38314,10 @@ public final class ClientProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -38327,12 +38327,12 @@ public final class ClientProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -38441,7 +38441,7 @@ public final class ClientProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < regionActionResult_.size(); i++) {
         output.writeMessage(1, regionActionResult_.get(i));
@@ -38461,15 +38461,15 @@ public final class ClientProtos {
 
       size = 0;
       for (int i = 0; i < regionActionResult_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, regionActionResult_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, processed_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getRegionStatistics());
       }
       size += unknownFields.getSerializedSize();
@@ -38518,7 +38518,7 @@ public final class ClientProtos {
       }
       if (hasProcessed()) {
         hash = (37 * hash) + PROCESSED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getProcessed());
       }
       if (hasRegionStatistics()) {
@@ -38531,61 +38531,61 @@ public final class ClientProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -38603,7 +38603,7 @@ public final class ClientProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -38611,15 +38611,15 @@ public final class ClientProtos {
      * Protobuf type {@code hbase.pb.MultiResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MultiResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -38632,12 +38632,12 @@ public final class ClientProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionActionResultFieldBuilder();
           getRegionStatisticsFieldBuilder();
@@ -38662,7 +38662,7 @@ public final class ClientProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.internal_static_hbase_pb_MultiResponse_descriptor;
       }
@@ -38713,29 +38713,29 @@ public final class ClientProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse)other);
         } else {
@@ -38765,7 +38765,7 @@ public final class ClientProtos {
               regionActionResult_ = other.regionActionResult_;
               bitField0_ = (bitField0_ & ~0x00000001);
               regionActionResultBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionActionResultFieldBuilder() : null;
             } else {
               regionActionResultBuilder_.addAllMessages(other.regionActionResult_);
@@ -38798,13 +38798,13 @@ public final class ClientProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -38825,7 +38825,7 @@ public final class ClientProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResultOrBuilder> regionActionResultBuilder_;
 
       /**
@@ -38957,7 +38957,7 @@ public final class ClientProtos {
           java.lang.Iterable values) {
         if (regionActionResultBuilder_ == null) {
           ensureRegionActionResultIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionActionResult_);
           onChanged();
         } else {
@@ -39041,11 +39041,11 @@ public final class ClientProtos {
            getRegionActionResultBuilderList() {
         return getRegionActionResultFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResultOrBuilder> 
           getRegionActionResultFieldBuilder() {
         if (regionActionResultBuilder_ == null) {
-          regionActionResultBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionActionResultBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResultOrBuilder>(
                   regionActionResult_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -39105,7 +39105,7 @@ public final class ClientProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats regionStatistics_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStatsOrBuilder> regionStatisticsBuilder_;
       /**
        * optional .hbase.pb.MultiRegionLoadStats regionStatistics = 3;
@@ -39208,11 +39208,11 @@ public final class ClientProtos {
       /**
        * optional .hbase.pb.MultiRegionLoadStats regionStatistics = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStatsOrBuilder> 
           getRegionStatisticsFieldBuilder() {
         if (regionStatisticsBuilder_ == null) {
-          regionStatisticsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionStatisticsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStatsOrBuilder>(
                   getRegionStatistics(),
                   getParentForChildren(),
@@ -39222,12 +39222,12 @@ public final class ClientProtos {
         return regionStatisticsBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -39245,22 +39245,22 @@ public final class ClientProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MultiResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MultiResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -39274,7 +39274,7 @@ public final class ClientProtos {
    * Protobuf service {@code hbase.pb.ClientService}
    */
   public static abstract class ClientService
-      implements com.google.protobuf.Service {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.Service {
     protected ClientService() {}
 
     public interface Interface {
@@ -39282,167 +39282,167 @@ public final class ClientProtos {
        * rpc Get(.hbase.pb.GetRequest) returns (.hbase.pb.GetResponse);
        */
       public abstract void get(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc Mutate(.hbase.pb.MutateRequest) returns (.hbase.pb.MutateResponse);
        */
       public abstract void mutate(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc Scan(.hbase.pb.ScanRequest) returns (.hbase.pb.ScanResponse);
        */
       public abstract void scan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc BulkLoadHFile(.hbase.pb.BulkLoadHFileRequest) returns (.hbase.pb.BulkLoadHFileResponse);
        */
       public abstract void bulkLoadHFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc PrepareBulkLoad(.hbase.pb.PrepareBulkLoadRequest) returns (.hbase.pb.PrepareBulkLoadResponse);
        */
       public abstract void prepareBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc CleanupBulkLoad(.hbase.pb.CleanupBulkLoadRequest) returns (.hbase.pb.CleanupBulkLoadResponse);
        */
       public abstract void cleanupBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc ExecService(.hbase.pb.CoprocessorServiceRequest) returns (.hbase.pb.CoprocessorServiceResponse);
        */
       public abstract void execService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc ExecRegionServerService(.hbase.pb.CoprocessorServiceRequest) returns (.hbase.pb.CoprocessorServiceResponse);
        */
       public abstract void execRegionServerService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc Multi(.hbase.pb.MultiRequest) returns (.hbase.pb.MultiResponse);
        */
       public abstract void multi(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     }
 
-    public static com.google.protobuf.Service newReflectiveService(
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
         final Interface impl) {
       return new ClientService() {
         @java.lang.Override
         public  void get(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.get(controller, request, done);
         }
 
         @java.lang.Override
         public  void mutate(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.mutate(controller, request, done);
         }
 
         @java.lang.Override
         public  void scan(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.scan(controller, request, done);
         }
 
         @java.lang.Override
         public  void bulkLoadHFile(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.bulkLoadHFile(controller, request, done);
         }
 
         @java.lang.Override
         public  void prepareBulkLoad(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.prepareBulkLoad(controller, request, done);
         }
 
         @java.lang.Override
         public  void cleanupBulkLoad(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.cleanupBulkLoad(controller, request, done);
         }
 
         @java.lang.Override
         public  void execService(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.execService(controller, request, done);
         }
 
         @java.lang.Override
         public  void execRegionServerService(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.execRegionServerService(controller, request, done);
         }
 
         @java.lang.Override
         public  void multi(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.multi(controller, request, done);
         }
 
       };
     }
 
-    public static com.google.protobuf.BlockingService
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
         newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService() {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
             getDescriptorForType() {
           return getDescriptor();
         }
 
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message callBlockingMethod(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.callBlockingMethod() given method descriptor for " +
@@ -39472,9 +39472,9 @@ public final class ClientProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getRequestPrototype() given method " +
@@ -39504,9 +39504,9 @@ public final class ClientProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getResponsePrototype() given method " +
@@ -39543,90 +39543,90 @@ public final class ClientProtos {
      * rpc Get(.hbase.pb.GetRequest) returns (.hbase.pb.GetResponse);
      */
     public abstract void get(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc Mutate(.hbase.pb.MutateRequest) returns (.hbase.pb.MutateResponse);
      */
     public abstract void mutate(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc Scan(.hbase.pb.ScanRequest) returns (.hbase.pb.ScanResponse);
      */
     public abstract void scan(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc BulkLoadHFile(.hbase.pb.BulkLoadHFileRequest) returns (.hbase.pb.BulkLoadHFileResponse);
      */
     public abstract void bulkLoadHFile(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc PrepareBulkLoad(.hbase.pb.PrepareBulkLoadRequest) returns (.hbase.pb.PrepareBulkLoadResponse);
      */
     public abstract void prepareBulkLoad(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc CleanupBulkLoad(.hbase.pb.CleanupBulkLoadRequest) returns (.hbase.pb.CleanupBulkLoadResponse);
      */
     public abstract void cleanupBulkLoad(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc ExecService(.hbase.pb.CoprocessorServiceRequest) returns (.hbase.pb.CoprocessorServiceResponse);
      */
     public abstract void execService(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc ExecRegionServerService(.hbase.pb.CoprocessorServiceRequest) returns (.hbase.pb.CoprocessorServiceResponse);
      */
     public abstract void execRegionServerService(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc Multi(.hbase.pb.MultiRequest) returns (.hbase.pb.MultiResponse);
      */
     public abstract void multi(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.getDescriptor().getServices().get(0);
     }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
 
     public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Message> done) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.callMethod() given method descriptor for wrong " +
@@ -39635,47 +39635,47 @@ public final class ClientProtos {
       switch(method.getIndex()) {
         case 0:
           this.get(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 1:
           this.mutate(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 2:
           this.scan(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 3:
           this.bulkLoadHFile(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 4:
           this.prepareBulkLoad(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 5:
           this.cleanupBulkLoad(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 6:
           this.execService(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 7:
           this.execRegionServerService(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 8:
           this.multi(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         default:
@@ -39683,9 +39683,9 @@ public final class ClientProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getRequestPrototype() given method " +
@@ -39715,9 +39715,9 @@ public final class ClientProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getResponsePrototype() given method " +
@@ -39748,151 +39748,151 @@ public final class ClientProtos {
     }
 
     public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
       return new Stub(channel);
     }
 
     public static final class Stub extends org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
+      private Stub(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.RpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel;
 
-      public com.google.protobuf.RpcChannel getChannel() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel getChannel() {
         return channel;
       }
 
       public  void get(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(0),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse.getDefaultInstance()));
       }
 
       public  void mutate(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance()));
       }
 
       public  void scan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(2),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance()));
       }
 
       public  void bulkLoadHFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(3),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance()));
       }
 
       public  void prepareBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(4),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse.getDefaultInstance()));
       }
 
       public  void cleanupBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(5),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse.getDefaultInstance()));
       }
 
       public  void execService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(6),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()));
       }
 
       public  void execRegionServerService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(7),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()));
       }
 
       public  void multi(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(8),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance()));
@@ -39900,68 +39900,68 @@ public final class ClientProtos {
     }
 
     public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
       return new BlockingStub(channel);
     }
 
     public interface BlockingInterface {
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse get(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse mutate(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse scan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse bulkLoadHFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse prepareBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse cleanupBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execRegionServerService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse multi(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+      private BlockingStub(org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.BlockingRpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse get(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(0),
           controller,
@@ -39971,9 +39971,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse mutate(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(1),
           controller,
@@ -39983,9 +39983,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse scan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(2),
           controller,
@@ -39995,9 +39995,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse bulkLoadHFile(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(3),
           controller,
@@ -40007,9 +40007,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse prepareBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(4),
           controller,
@@ -40019,9 +40019,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse cleanupBulkLoad(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(5),
           controller,
@@ -40031,9 +40031,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(6),
           controller,
@@ -40043,9 +40043,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execRegionServerService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(7),
           controller,
@@ -40055,9 +40055,9 @@ public final class ClientProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse multi(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(8),
           controller,
@@ -40070,192 +40070,192 @@ public final class ClientProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.ClientService)
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Authorizations_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Authorizations_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CellVisibility_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CellVisibility_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Column_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Column_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Get_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Get_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Result_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Result_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Condition_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Condition_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MutationProto_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MutationProto_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MutationProto_ColumnValue_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MutationProto_ColumnValue_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MutateRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MutateRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MutateResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MutateResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Scan_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Scan_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ScanRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ScanRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ScanResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ScanResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BulkLoadHFileRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BulkLoadHFileResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BulkLoadHFileResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DelegationToken_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DelegationToken_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_PrepareBulkLoadRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_PrepareBulkLoadRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_PrepareBulkLoadResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_PrepareBulkLoadResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CleanupBulkLoadRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CleanupBulkLoadRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CleanupBulkLoadResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CleanupBulkLoadResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CoprocessorServiceCall_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CoprocessorServiceCall_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CoprocessorServiceResult_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CoprocessorServiceResult_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CoprocessorServiceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CoprocessorServiceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CoprocessorServiceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CoprocessorServiceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Action_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Action_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionAction_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionAction_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionLoadStats_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionLoadStats_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MultiRegionLoadStats_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MultiRegionLoadStats_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ResultOrException_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ResultOrException_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionActionResult_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionActionResult_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MultiRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MultiRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MultiResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MultiResponse_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -40418,17 +40418,17 @@ public final class ClientProtos {
       "seBI\n1org.apache.hadoop.hbase.shaded.pro" +
       "tobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos.getDescriptor(),
@@ -40438,217 +40438,217 @@ public final class ClientProtos {
     internal_static_hbase_pb_Authorizations_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_Authorizations_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Authorizations_descriptor,
         new java.lang.String[] { "Label", });
     internal_static_hbase_pb_CellVisibility_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_CellVisibility_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CellVisibility_descriptor,
         new java.lang.String[] { "Expression", });
     internal_static_hbase_pb_Column_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_Column_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Column_descriptor,
         new java.lang.String[] { "Family", "Qualifier", });
     internal_static_hbase_pb_Get_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_Get_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Get_descriptor,
         new java.lang.String[] { "Row", "Column", "Attribute", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "StoreLimit", "StoreOffset", "ExistenceOnly", "Consistency", "CfTimeRange", });
     internal_static_hbase_pb_Result_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_Result_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Result_descriptor,
         new java.lang.String[] { "Cell", "AssociatedCellCount", "Exists", "Stale", "Partial", });
     internal_static_hbase_pb_GetRequest_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_GetRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetRequest_descriptor,
         new java.lang.String[] { "Region", "Get", });
     internal_static_hbase_pb_GetResponse_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_GetResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetResponse_descriptor,
         new java.lang.String[] { "Result", });
     internal_static_hbase_pb_Condition_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_Condition_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Condition_descriptor,
         new java.lang.String[] { "Row", "Family", "Qualifier", "CompareType", "Comparator", });
     internal_static_hbase_pb_MutationProto_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_MutationProto_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MutationProto_descriptor,
         new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Timestamp", "Attribute", "Durability", "TimeRange", "AssociatedCellCount", "Nonce", });
     internal_static_hbase_pb_MutationProto_ColumnValue_descriptor =
       internal_static_hbase_pb_MutationProto_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_MutationProto_ColumnValue_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MutationProto_ColumnValue_descriptor,
         new java.lang.String[] { "Family", "QualifierValue", });
     internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_descriptor =
       internal_static_hbase_pb_MutationProto_ColumnValue_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MutationProto_ColumnValue_QualifierValue_descriptor,
         new java.lang.String[] { "Qualifier", "Value", "Timestamp", "DeleteType", "Tags", });
     internal_static_hbase_pb_MutateRequest_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_MutateRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MutateRequest_descriptor,
         new java.lang.String[] { "Region", "Mutation", "Condition", "NonceGroup", });
     internal_static_hbase_pb_MutateResponse_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_MutateResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MutateResponse_descriptor,
         new java.lang.String[] { "Result", "Processed", });
     internal_static_hbase_pb_Scan_descriptor =
       getDescriptor().getMessageTypes().get(11);
     internal_static_hbase_pb_Scan_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Scan_descriptor,
         new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", });
     internal_static_hbase_pb_ScanRequest_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ScanRequest_descriptor,
         new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", });
     internal_static_hbase_pb_ScanResponse_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ScanResponse_descriptor,
         new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", });
     internal_static_hbase_pb_BulkLoadHFileRequest_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BulkLoadHFileRequest_descriptor,
         new java.lang.String[] { "Region", "FamilyPath", "AssignSeqNum", "FsToken", "BulkToken", "CopyFile", });
     internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor =
       internal_static_hbase_pb_BulkLoadHFileRequest_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor,
         new java.lang.String[] { "Family", "Path", });
     internal_static_hbase_pb_BulkLoadHFileResponse_descriptor =
       getDescriptor().getMessageTypes().get(15);
     internal_static_hbase_pb_BulkLoadHFileResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BulkLoadHFileResponse_descriptor,
         new java.lang.String[] { "Loaded", });
     internal_static_hbase_pb_DelegationToken_descriptor =
       getDescriptor().getMessageTypes().get(16);
     internal_static_hbase_pb_DelegationToken_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DelegationToken_descriptor,
         new java.lang.String[] { "Identifier", "Password", "Kind", "Service", });
     internal_static_hbase_pb_PrepareBulkLoadRequest_descriptor =
       getDescriptor().getMessageTypes().get(17);
     internal_static_hbase_pb_PrepareBulkLoadRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_PrepareBulkLoadRequest_descriptor,
         new java.lang.String[] { "TableName", "Region", });
     internal_static_hbase_pb_PrepareBulkLoadResponse_descriptor =
       getDescriptor().getMessageTypes().get(18);
     internal_static_hbase_pb_PrepareBulkLoadResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_PrepareBulkLoadResponse_descriptor,
         new java.lang.String[] { "BulkToken", });
     internal_static_hbase_pb_CleanupBulkLoadRequest_descriptor =
       getDescriptor().getMessageTypes().get(19);
     internal_static_hbase_pb_CleanupBulkLoadRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CleanupBulkLoadRequest_descriptor,
         new java.lang.String[] { "BulkToken", "Region", });
     internal_static_hbase_pb_CleanupBulkLoadResponse_descriptor =
       getDescriptor().getMessageTypes().get(20);
     internal_static_hbase_pb_CleanupBulkLoadResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CleanupBulkLoadResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_CoprocessorServiceCall_descriptor =
       getDescriptor().getMessageTypes().get(21);
     internal_static_hbase_pb_CoprocessorServiceCall_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CoprocessorServiceCall_descriptor,
         new java.lang.String[] { "Row", "ServiceName", "MethodName", "Request", });
     internal_static_hbase_pb_CoprocessorServiceResult_descriptor =
       getDescriptor().getMessageTypes().get(22);
     internal_static_hbase_pb_CoprocessorServiceResult_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CoprocessorServiceResult_descriptor,
         new java.lang.String[] { "Value", });
     internal_static_hbase_pb_CoprocessorServiceRequest_descriptor =
       getDescriptor().getMessageTypes().get(23);
     internal_static_hbase_pb_CoprocessorServiceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CoprocessorServiceRequest_descriptor,
         new java.lang.String[] { "Region", "Call", });
     internal_static_hbase_pb_CoprocessorServiceResponse_descriptor =
       getDescriptor().getMessageTypes().get(24);
     internal_static_hbase_pb_CoprocessorServiceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CoprocessorServiceResponse_descriptor,
         new java.lang.String[] { "Region", "Value", });
     internal_static_hbase_pb_Action_descriptor =
       getDescriptor().getMessageTypes().get(25);
     internal_static_hbase_pb_Action_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Action_descriptor,
         new java.lang.String[] { "Index", "Mutation", "Get", "ServiceCall", });
     internal_static_hbase_pb_RegionAction_descriptor =
       getDescriptor().getMessageTypes().get(26);
     internal_static_hbase_pb_RegionAction_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionAction_descriptor,
         new java.lang.String[] { "Region", "Atomic", "Action", });
     internal_static_hbase_pb_RegionLoadStats_descriptor =
       getDescriptor().getMessageTypes().get(27);
     internal_static_hbase_pb_RegionLoadStats_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionLoadStats_descriptor,
         new java.lang.String[] { "MemstoreLoad", "HeapOccupancy", "CompactionPressure", });
     internal_static_hbase_pb_MultiRegionLoadStats_descriptor =
       getDescriptor().getMessageTypes().get(28);
     internal_static_hbase_pb_MultiRegionLoadStats_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MultiRegionLoadStats_descriptor,
         new java.lang.String[] { "Region", "Stat", });
     internal_static_hbase_pb_ResultOrException_descriptor =
       getDescriptor().getMessageTypes().get(29);
     internal_static_hbase_pb_ResultOrException_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ResultOrException_descriptor,
         new java.lang.String[] { "Index", "Result", "Exception", "ServiceResult", "LoadStats", });
     internal_static_hbase_pb_RegionActionResult_descriptor =
       getDescriptor().getMessageTypes().get(30);
     internal_static_hbase_pb_RegionActionResult_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionActionResult_descriptor,
         new java.lang.String[] { "ResultOrException", "Exception", });
     internal_static_hbase_pb_MultiRequest_descriptor =
       getDescriptor().getMessageTypes().get(31);
     internal_static_hbase_pb_MultiRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MultiRequest_descriptor,
         new java.lang.String[] { "RegionAction", "NonceGroup", "Condition", });
     internal_static_hbase_pb_MultiResponse_descriptor =
       getDescriptor().getMessageTypes().get(32);
     internal_static_hbase_pb_MultiResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MultiResponse_descriptor,
         new java.lang.String[] { "RegionActionResult", "Processed", "RegionStatistics", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
index bd132af..326f217 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ClusterIdProtos {
   private ClusterIdProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface ClusterIdOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ClusterId)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -41,7 +41,7 @@ public final class ClusterIdProtos {
      *
      * required string cluster_id = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClusterIdBytes();
   }
   /**
@@ -54,11 +54,11 @@ public final class ClusterIdProtos {
    * Protobuf type {@code hbase.pb.ClusterId}
    */
   public  static final class ClusterId extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ClusterId)
       ClusterIdOrBuilder {
     // Use ClusterId.newBuilder() to construct.
-    private ClusterId(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ClusterId(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ClusterId() {
@@ -66,18 +66,18 @@ public final class ClusterIdProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ClusterId(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -94,29 +94,29 @@ public final class ClusterIdProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               clusterId_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.internal_static_hbase_pb_ClusterId_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.internal_static_hbase_pb_ClusterId_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -148,8 +148,8 @@ public final class ClusterIdProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           clusterId_ = s;
@@ -164,17 +164,17 @@ public final class ClusterIdProtos {
      *
      * required string cluster_id = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClusterIdBytes() {
       java.lang.Object ref = clusterId_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         clusterId_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -192,10 +192,10 @@ public final class ClusterIdProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterId_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterId_);
       }
       unknownFields.writeTo(output);
     }
@@ -206,7 +206,7 @@ public final class ClusterIdProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterId_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterId_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -251,61 +251,61 @@ public final class ClusterIdProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -323,7 +323,7 @@ public final class ClusterIdProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -337,15 +337,15 @@ public final class ClusterIdProtos {
      * Protobuf type {@code hbase.pb.ClusterId}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ClusterId)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterIdOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.internal_static_hbase_pb_ClusterId_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.internal_static_hbase_pb_ClusterId_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -358,12 +358,12 @@ public final class ClusterIdProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -374,7 +374,7 @@ public final class ClusterIdProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.internal_static_hbase_pb_ClusterId_descriptor;
       }
@@ -408,29 +408,29 @@ public final class ClusterIdProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId)other);
         } else {
@@ -459,13 +459,13 @@ public final class ClusterIdProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -498,8 +498,8 @@ public final class ClusterIdProtos {
       public java.lang.String getClusterId() {
         java.lang.Object ref = clusterId_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             clusterId_ = s;
@@ -516,17 +516,17 @@ public final class ClusterIdProtos {
        *
        * required string cluster_id = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getClusterIdBytes() {
         java.lang.Object ref = clusterId_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           clusterId_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -567,7 +567,7 @@ public final class ClusterIdProtos {
        * required string cluster_id = 1;
        */
       public Builder setClusterIdBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -577,12 +577,12 @@ public final class ClusterIdProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -600,22 +600,22 @@ public final class ClusterIdProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ClusterId parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ClusterId(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -625,17 +625,17 @@ public final class ClusterIdProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ClusterId_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ClusterId_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -644,22 +644,22 @@ public final class ClusterIdProtos {
       "p.hbase.shaded.protobuf.generatedB\017Clust" +
       "erIdProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_ClusterId_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_ClusterId_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ClusterId_descriptor,
         new java.lang.String[] { "ClusterId", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
index f945184..97c6d05 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ClusterStatusProtos {
   private ClusterStatusProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface RegionStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionInfo region_info = 1;
@@ -53,11 +53,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.RegionState}
    */
   public  static final class RegionState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionState)
       RegionStateOrBuilder {
     // Use RegionState.newBuilder() to construct.
-    private RegionState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionState() {
@@ -66,18 +66,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -124,22 +124,22 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -150,7 +150,7 @@ public final class ClusterStatusProtos {
      * Protobuf enum {@code hbase.pb.RegionState.State}
      */
     public enum State
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * 
        * region is in an offline state
@@ -430,27 +430,27 @@ public final class ClusterStatusProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           State> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public State findValueByNumber(int number) {
                 return State.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState.getDescriptor().getEnumTypes().get(0);
       }
@@ -458,7 +458,7 @@ public final class ClusterStatusProtos {
       private static final State[] VALUES = values();
 
       public static State valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -550,7 +550,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegionInfo());
@@ -570,15 +570,15 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegionInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, state_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, stamp_);
       }
       size += unknownFields.getSerializedSize();
@@ -633,7 +633,7 @@ public final class ClusterStatusProtos {
       }
       if (hasStamp()) {
         hash = (37 * hash) + STAMP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getStamp());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -642,61 +642,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -714,7 +714,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -722,15 +722,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.RegionState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -743,12 +743,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionInfoFieldBuilder();
         }
@@ -768,7 +768,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionState_descriptor;
       }
@@ -814,29 +814,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState)other);
         } else {
@@ -875,13 +875,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -894,7 +894,7 @@ public final class ClusterStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
       /**
        * required .hbase.pb.RegionInfo region_info = 1;
@@ -997,11 +997,11 @@ public final class ClusterStatusProtos {
       /**
        * required .hbase.pb.RegionInfo region_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   getRegionInfo(),
                   getParentForChildren(),
@@ -1079,12 +1079,12 @@ public final class ClusterStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1102,22 +1102,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1129,7 +1129,7 @@ public final class ClusterStatusProtos {
 
   public interface RegionInTransitionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionInTransition)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier spec = 1;
@@ -1161,29 +1161,29 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.RegionInTransition}
    */
   public  static final class RegionInTransition extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionInTransition)
       RegionInTransitionOrBuilder {
     // Use RegionInTransition.newBuilder() to construct.
-    private RegionInTransition(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionInTransition(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionInTransition() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionInTransition(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1227,22 +1227,22 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionInTransition_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionInTransition_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1318,7 +1318,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSpec());
@@ -1335,11 +1335,11 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSpec());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getRegionState());
       }
       size += unknownFields.getSerializedSize();
@@ -1394,61 +1394,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1466,7 +1466,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1474,15 +1474,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.RegionInTransition}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionInTransition)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransitionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionInTransition_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionInTransition_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1495,12 +1495,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSpecFieldBuilder();
           getRegionStateFieldBuilder();
@@ -1523,7 +1523,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionInTransition_descriptor;
       }
@@ -1569,29 +1569,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition)other);
         } else {
@@ -1630,13 +1630,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1649,7 +1649,7 @@ public final class ClusterStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier spec_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> specBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier spec = 1;
@@ -1752,11 +1752,11 @@ public final class ClusterStatusProtos {
       /**
        * required .hbase.pb.RegionSpecifier spec = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getSpecFieldBuilder() {
         if (specBuilder_ == null) {
-          specBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          specBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getSpec(),
                   getParentForChildren(),
@@ -1767,7 +1767,7 @@ public final class ClusterStatusProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState regionState_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStateOrBuilder> regionStateBuilder_;
       /**
        * required .hbase.pb.RegionState region_state = 2;
@@ -1870,11 +1870,11 @@ public final class ClusterStatusProtos {
       /**
        * required .hbase.pb.RegionState region_state = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStateOrBuilder> 
           getRegionStateFieldBuilder() {
         if (regionStateBuilder_ == null) {
-          regionStateBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionStateBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStateOrBuilder>(
                   getRegionState(),
                   getParentForChildren(),
@@ -1884,12 +1884,12 @@ public final class ClusterStatusProtos {
         return regionStateBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1907,22 +1907,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionInTransition parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionInTransition(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1934,7 +1934,7 @@ public final class ClusterStatusProtos {
 
   public interface StoreSequenceIdOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StoreSequenceId)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes family_name = 1;
@@ -1943,7 +1943,7 @@ public final class ClusterStatusProtos {
     /**
      * required bytes family_name = 1;
      */
-    com.google.protobuf.ByteString getFamilyName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName();
 
     /**
      * required uint64 sequence_id = 2;
@@ -1963,31 +1963,31 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.StoreSequenceId}
    */
   public  static final class StoreSequenceId extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StoreSequenceId)
       StoreSequenceIdOrBuilder {
     // Use StoreSequenceId.newBuilder() to construct.
-    private StoreSequenceId(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StoreSequenceId(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StoreSequenceId() {
-      familyName_ = com.google.protobuf.ByteString.EMPTY;
+      familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       sequenceId_ = 0L;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StoreSequenceId(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2015,22 +2015,22 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_StoreSequenceId_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_StoreSequenceId_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2039,7 +2039,7 @@ public final class ClusterStatusProtos {
 
     private int bitField0_;
     public static final int FAMILY_NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString familyName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_;
     /**
      * required bytes family_name = 1;
      */
@@ -2049,7 +2049,7 @@ public final class ClusterStatusProtos {
     /**
      * required bytes family_name = 1;
      */
-    public com.google.protobuf.ByteString getFamilyName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
       return familyName_;
     }
 
@@ -2086,7 +2086,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, familyName_);
@@ -2103,11 +2103,11 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, familyName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, sequenceId_);
       }
       size += unknownFields.getSerializedSize();
@@ -2154,7 +2154,7 @@ public final class ClusterStatusProtos {
       }
       if (hasSequenceId()) {
         hash = (37 * hash) + SEQUENCE_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getSequenceId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -2163,61 +2163,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2235,7 +2235,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2248,15 +2248,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.StoreSequenceId}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StoreSequenceId)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_StoreSequenceId_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_StoreSequenceId_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2269,25 +2269,25 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        familyName_ = com.google.protobuf.ByteString.EMPTY;
+        familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         sequenceId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_StoreSequenceId_descriptor;
       }
@@ -2325,29 +2325,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId)other);
         } else {
@@ -2380,13 +2380,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2398,7 +2398,7 @@ public final class ClusterStatusProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family_name = 1;
        */
@@ -2408,13 +2408,13 @@ public final class ClusterStatusProtos {
       /**
        * required bytes family_name = 1;
        */
-      public com.google.protobuf.ByteString getFamilyName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
         return familyName_;
       }
       /**
        * required bytes family_name = 1;
        */
-      public Builder setFamilyName(com.google.protobuf.ByteString value) {
+      public Builder setFamilyName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2465,12 +2465,12 @@ public final class ClusterStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2488,22 +2488,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StoreSequenceId parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StoreSequenceId(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2515,7 +2515,7 @@ public final class ClusterStatusProtos {
 
   public interface RegionStoreSequenceIdsOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionStoreSequenceIds)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 last_flushed_sequence_id = 1;
@@ -2560,11 +2560,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.RegionStoreSequenceIds}
    */
   public  static final class RegionStoreSequenceIds extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionStoreSequenceIds)
       RegionStoreSequenceIdsOrBuilder {
     // Use RegionStoreSequenceIds.newBuilder() to construct.
-    private RegionStoreSequenceIds(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionStoreSequenceIds(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionStoreSequenceIds() {
@@ -2573,18 +2573,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionStoreSequenceIds(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2616,10 +2616,10 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -2629,12 +2629,12 @@ public final class ClusterStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionStoreSequenceIds_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionStoreSequenceIds_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2712,7 +2712,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, lastFlushedSequenceId_);
@@ -2729,11 +2729,11 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, lastFlushedSequenceId_);
       }
       for (int i = 0; i < storeSequenceId_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, storeSequenceId_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -2773,7 +2773,7 @@ public final class ClusterStatusProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLastFlushedSequenceId()) {
         hash = (37 * hash) + LAST_FLUSHED_SEQUENCE_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastFlushedSequenceId());
       }
       if (getStoreSequenceIdCount() > 0) {
@@ -2786,61 +2786,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2858,7 +2858,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2872,15 +2872,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.RegionStoreSequenceIds}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionStoreSequenceIds)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIdsOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionStoreSequenceIds_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionStoreSequenceIds_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2893,12 +2893,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getStoreSequenceIdFieldBuilder();
         }
@@ -2916,7 +2916,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionStoreSequenceIds_descriptor;
       }
@@ -2959,29 +2959,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds)other);
         } else {
@@ -3014,7 +3014,7 @@ public final class ClusterStatusProtos {
               storeSequenceId_ = other.storeSequenceId_;
               bitField0_ = (bitField0_ & ~0x00000002);
               storeSequenceIdBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStoreSequenceIdFieldBuilder() : null;
             } else {
               storeSequenceIdBuilder_.addAllMessages(other.storeSequenceId_);
@@ -3039,13 +3039,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3098,7 +3098,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder> storeSequenceIdBuilder_;
 
       /**
@@ -3230,7 +3230,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (storeSequenceIdBuilder_ == null) {
           ensureStoreSequenceIdIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, storeSequenceId_);
           onChanged();
         } else {
@@ -3314,11 +3314,11 @@ public final class ClusterStatusProtos {
            getStoreSequenceIdBuilderList() {
         return getStoreSequenceIdFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder> 
           getStoreSequenceIdFieldBuilder() {
         if (storeSequenceIdBuilder_ == null) {
-          storeSequenceIdBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          storeSequenceIdBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder>(
                   storeSequenceId_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -3329,12 +3329,12 @@ public final class ClusterStatusProtos {
         return storeSequenceIdBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3352,22 +3352,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionStoreSequenceIds parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionStoreSequenceIds(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3379,7 +3379,7 @@ public final class ClusterStatusProtos {
 
   public interface RegionLoadOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionLoad)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -3743,11 +3743,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.RegionLoad}
    */
   public  static final class RegionLoad extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionLoad)
       RegionLoadOrBuilder {
     // Use RegionLoad.newBuilder() to construct.
-    private RegionLoad(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionLoad(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionLoad() {
@@ -3772,18 +3772,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionLoad(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3908,10 +3908,10 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00020000) == 0x00020000)) {
@@ -3921,12 +3921,12 @@ public final class ClusterStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionLoad_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionLoad_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4437,7 +4437,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegionSpecifier());
@@ -4505,79 +4505,79 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegionSpecifier());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, stores_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(3, storefiles_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(4, storeUncompressedSizeMB_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(5, storefileSizeMB_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(6, memstoreSizeMB_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(7, storefileIndexSizeMB_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(8, readRequestsCount_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(9, writeRequestsCount_);
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(10, totalCompactingKVs_);
       }
       if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(11, currentCompactedKVs_);
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(12, rootIndexSizeKB_);
       }
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(13, totalStaticIndexSizeKB_);
       }
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(14, totalStaticBloomSizeKB_);
       }
       if (((bitField0_ & 0x00004000) == 0x00004000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(15, completeSequenceId_);
       }
       if (((bitField0_ & 0x00008000) == 0x00008000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeFloatSize(16, dataLocality_);
       }
       if (((bitField0_ & 0x00010000) == 0x00010000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(17, lastMajorCompactionTs_);
       }
       for (int i = 0; i < storeCompleteSequenceId_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(18, storeCompleteSequenceId_.get(i));
       }
       if (((bitField0_ & 0x00020000) == 0x00020000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(19, filteredReadRequestsCount_);
       }
       size += unknownFields.getSerializedSize();
@@ -4732,22 +4732,22 @@ public final class ClusterStatusProtos {
       }
       if (hasReadRequestsCount()) {
         hash = (37 * hash) + READ_REQUESTS_COUNT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getReadRequestsCount());
       }
       if (hasWriteRequestsCount()) {
         hash = (37 * hash) + WRITE_REQUESTS_COUNT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getWriteRequestsCount());
       }
       if (hasTotalCompactingKVs()) {
         hash = (37 * hash) + TOTAL_COMPACTING_KVS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTotalCompactingKVs());
       }
       if (hasCurrentCompactedKVs()) {
         hash = (37 * hash) + CURRENT_COMPACTED_KVS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getCurrentCompactedKVs());
       }
       if (hasRootIndexSizeKB()) {
@@ -4764,7 +4764,7 @@ public final class ClusterStatusProtos {
       }
       if (hasCompleteSequenceId()) {
         hash = (37 * hash) + COMPLETE_SEQUENCE_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getCompleteSequenceId());
       }
       if (hasDataLocality()) {
@@ -4774,7 +4774,7 @@ public final class ClusterStatusProtos {
       }
       if (hasLastMajorCompactionTs()) {
         hash = (37 * hash) + LAST_MAJOR_COMPACTION_TS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastMajorCompactionTs());
       }
       if (getStoreCompleteSequenceIdCount() > 0) {
@@ -4783,7 +4783,7 @@ public final class ClusterStatusProtos {
       }
       if (hasFilteredReadRequestsCount()) {
         hash = (37 * hash) + FILTERED_READ_REQUESTS_COUNT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getFilteredReadRequestsCount());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -4792,61 +4792,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4864,7 +4864,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4872,15 +4872,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.RegionLoad}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionLoad)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionLoad_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionLoad_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4893,12 +4893,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionSpecifierFieldBuilder();
           getStoreCompleteSequenceIdFieldBuilder();
@@ -4955,7 +4955,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_RegionLoad_descriptor;
       }
@@ -5070,29 +5070,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad)other);
         } else {
@@ -5173,7 +5173,7 @@ public final class ClusterStatusProtos {
               storeCompleteSequenceId_ = other.storeCompleteSequenceId_;
               bitField0_ = (bitField0_ & ~0x00020000);
               storeCompleteSequenceIdBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStoreCompleteSequenceIdFieldBuilder() : null;
             } else {
               storeCompleteSequenceIdBuilder_.addAllMessages(other.storeCompleteSequenceId_);
@@ -5204,13 +5204,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5223,7 +5223,7 @@ public final class ClusterStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionSpecifier_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionSpecifierBuilder_;
       /**
        * 
@@ -5362,11 +5362,11 @@ public final class ClusterStatusProtos {
        *
        * required .hbase.pb.RegionSpecifier region_specifier = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionSpecifierFieldBuilder() {
         if (regionSpecifierBuilder_ == null) {
-          regionSpecifierBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionSpecifierBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegionSpecifier(),
                   getParentForChildren(),
@@ -6153,7 +6153,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder> storeCompleteSequenceIdBuilder_;
 
       /**
@@ -6325,7 +6325,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (storeCompleteSequenceIdBuilder_ == null) {
           ensureStoreCompleteSequenceIdIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, storeCompleteSequenceId_);
           onChanged();
         } else {
@@ -6441,11 +6441,11 @@ public final class ClusterStatusProtos {
            getStoreCompleteSequenceIdBuilderList() {
         return getStoreCompleteSequenceIdFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder> 
           getStoreCompleteSequenceIdFieldBuilder() {
         if (storeCompleteSequenceIdBuilder_ == null) {
-          storeCompleteSequenceIdBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          storeCompleteSequenceIdBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder>(
                   storeCompleteSequenceId_,
                   ((bitField0_ & 0x00020000) == 0x00020000),
@@ -6504,12 +6504,12 @@ public final class ClusterStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6527,22 +6527,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionLoad parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionLoad(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6554,7 +6554,7 @@ public final class ClusterStatusProtos {
 
   public interface ReplicationLoadSinkOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationLoadSink)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 ageOfLastAppliedOp = 1;
@@ -6578,11 +6578,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.ReplicationLoadSink}
    */
   public  static final class ReplicationLoadSink extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationLoadSink)
       ReplicationLoadSinkOrBuilder {
     // Use ReplicationLoadSink.newBuilder() to construct.
-    private ReplicationLoadSink(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicationLoadSink(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicationLoadSink() {
@@ -6591,18 +6591,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicationLoadSink(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6630,22 +6630,22 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSink_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6701,7 +6701,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, ageOfLastAppliedOp_);
@@ -6718,11 +6718,11 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, ageOfLastAppliedOp_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, timeStampsOfLastAppliedOp_);
       }
       size += unknownFields.getSerializedSize();
@@ -6765,12 +6765,12 @@ public final class ClusterStatusProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasAgeOfLastAppliedOp()) {
         hash = (37 * hash) + AGEOFLASTAPPLIEDOP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getAgeOfLastAppliedOp());
       }
       if (hasTimeStampsOfLastAppliedOp()) {
         hash = (37 * hash) + TIMESTAMPSOFLASTAPPLIEDOP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTimeStampsOfLastAppliedOp());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -6779,61 +6779,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6851,7 +6851,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6859,15 +6859,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.ReplicationLoadSink}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationLoadSink)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSink_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6880,12 +6880,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -6898,7 +6898,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSink_descriptor;
       }
@@ -6936,29 +6936,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink)other);
         } else {
@@ -6991,13 +6991,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7073,12 +7073,12 @@ public final class ClusterStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7096,22 +7096,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicationLoadSink parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicationLoadSink(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7123,7 +7123,7 @@ public final class ClusterStatusProtos {
 
   public interface ReplicationLoadSourceOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationLoadSource)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string peerID = 1;
@@ -7136,7 +7136,7 @@ public final class ClusterStatusProtos {
     /**
      * required string peerID = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getPeerIDBytes();
 
     /**
@@ -7179,11 +7179,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.ReplicationLoadSource}
    */
   public  static final class ReplicationLoadSource extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationLoadSource)
       ReplicationLoadSourceOrBuilder {
     // Use ReplicationLoadSource.newBuilder() to construct.
-    private ReplicationLoadSource(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicationLoadSource(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicationLoadSource() {
@@ -7195,18 +7195,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicationLoadSource(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7223,7 +7223,7 @@ public final class ClusterStatusProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               peerID_ = bs;
               break;
@@ -7250,22 +7250,22 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSource_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSource_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7289,8 +7289,8 @@ public final class ClusterStatusProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           peerID_ = s;
@@ -7301,17 +7301,17 @@ public final class ClusterStatusProtos {
     /**
      * required string peerID = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getPeerIDBytes() {
       java.lang.Object ref = peerID_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         peerID_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -7405,10 +7405,10 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, peerID_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, peerID_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt64(2, ageOfLastShippedOp_);
@@ -7431,22 +7431,22 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, peerID_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, peerID_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, ageOfLastShippedOp_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(3, sizeOfLogQueue_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, timeStampOfLastShippedOp_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(5, replicationLag_);
       }
       size += unknownFields.getSerializedSize();
@@ -7508,7 +7508,7 @@ public final class ClusterStatusProtos {
       }
       if (hasAgeOfLastShippedOp()) {
         hash = (37 * hash) + AGEOFLASTSHIPPEDOP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getAgeOfLastShippedOp());
       }
       if (hasSizeOfLogQueue()) {
@@ -7517,12 +7517,12 @@ public final class ClusterStatusProtos {
       }
       if (hasTimeStampOfLastShippedOp()) {
         hash = (37 * hash) + TIMESTAMPOFLASTSHIPPEDOP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTimeStampOfLastShippedOp());
       }
       if (hasReplicationLag()) {
         hash = (37 * hash) + REPLICATIONLAG_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getReplicationLag());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -7531,61 +7531,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7603,7 +7603,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7611,15 +7611,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.ReplicationLoadSource}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationLoadSource)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSource_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSource_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7632,12 +7632,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7656,7 +7656,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ReplicationLoadSource_descriptor;
       }
@@ -7706,29 +7706,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource)other);
         } else {
@@ -7781,13 +7781,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7812,8 +7812,8 @@ public final class ClusterStatusProtos {
       public java.lang.String getPeerID() {
         java.lang.Object ref = peerID_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             peerID_ = s;
@@ -7826,17 +7826,17 @@ public final class ClusterStatusProtos {
       /**
        * required string peerID = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getPeerIDBytes() {
         java.lang.Object ref = peerID_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           peerID_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -7865,7 +7865,7 @@ public final class ClusterStatusProtos {
        * required string peerID = 1;
        */
       public Builder setPeerIDBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8003,12 +8003,12 @@ public final class ClusterStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8026,22 +8026,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicationLoadSource parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicationLoadSource(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8053,7 +8053,7 @@ public final class ClusterStatusProtos {
 
   public interface ServerLoadOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ServerLoad)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -8372,11 +8372,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.ServerLoad}
    */
   public  static final class ServerLoad extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ServerLoad)
       ServerLoadOrBuilder {
     // Use ServerLoad.newBuilder() to construct.
-    private ServerLoad(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ServerLoad(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ServerLoad() {
@@ -8393,18 +8393,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ServerLoad(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8497,10 +8497,10 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
@@ -8516,12 +8516,12 @@ public final class ClusterStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ServerLoad_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ServerLoad_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8959,7 +8959,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, numberOfRequests_);
@@ -9003,47 +9003,47 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, numberOfRequests_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, totalNumberOfRequests_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(3, usedHeapMB_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(4, maxHeapMB_);
       }
       for (int i = 0; i < regionLoads_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, regionLoads_.get(i));
       }
       for (int i = 0; i < coprocessors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, coprocessors_.get(i));
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(7, reportStartTime_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(8, reportEndTime_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(9, infoServerPort_);
       }
       for (int i = 0; i < replLoadSource_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(10, replLoadSource_.get(i));
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(11, getReplLoadSink());
       }
       size += unknownFields.getSerializedSize();
@@ -9122,12 +9122,12 @@ public final class ClusterStatusProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasNumberOfRequests()) {
         hash = (37 * hash) + NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNumberOfRequests());
       }
       if (hasTotalNumberOfRequests()) {
         hash = (37 * hash) + TOTAL_NUMBER_OF_REQUESTS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTotalNumberOfRequests());
       }
       if (hasUsedHeapMB()) {
@@ -9148,12 +9148,12 @@ public final class ClusterStatusProtos {
       }
       if (hasReportStartTime()) {
         hash = (37 * hash) + REPORT_START_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getReportStartTime());
       }
       if (hasReportEndTime()) {
         hash = (37 * hash) + REPORT_END_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getReportEndTime());
       }
       if (hasInfoServerPort()) {
@@ -9174,61 +9174,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9246,7 +9246,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9254,15 +9254,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.ServerLoad}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ServerLoad)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ServerLoad_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ServerLoad_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9275,12 +9275,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionLoadsFieldBuilder();
           getCoprocessorsFieldBuilder();
@@ -9331,7 +9331,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ServerLoad_descriptor;
       }
@@ -9424,29 +9424,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad)other);
         } else {
@@ -9488,7 +9488,7 @@ public final class ClusterStatusProtos {
               regionLoads_ = other.regionLoads_;
               bitField0_ = (bitField0_ & ~0x00000010);
               regionLoadsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionLoadsFieldBuilder() : null;
             } else {
               regionLoadsBuilder_.addAllMessages(other.regionLoads_);
@@ -9514,7 +9514,7 @@ public final class ClusterStatusProtos {
               coprocessors_ = other.coprocessors_;
               bitField0_ = (bitField0_ & ~0x00000020);
               coprocessorsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getCoprocessorsFieldBuilder() : null;
             } else {
               coprocessorsBuilder_.addAllMessages(other.coprocessors_);
@@ -9549,7 +9549,7 @@ public final class ClusterStatusProtos {
               replLoadSource_ = other.replLoadSource_;
               bitField0_ = (bitField0_ & ~0x00000200);
               replLoadSourceBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getReplLoadSourceFieldBuilder() : null;
             } else {
               replLoadSourceBuilder_.addAllMessages(other.replLoadSource_);
@@ -9589,13 +9589,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9808,7 +9808,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> regionLoadsBuilder_;
 
       /**
@@ -9980,7 +9980,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (regionLoadsBuilder_ == null) {
           ensureRegionLoadsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionLoads_);
           onChanged();
         } else {
@@ -10096,11 +10096,11 @@ public final class ClusterStatusProtos {
            getRegionLoadsBuilderList() {
         return getRegionLoadsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder> 
           getRegionLoadsFieldBuilder() {
         if (regionLoadsBuilder_ == null) {
-          regionLoadsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionLoadsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoadOrBuilder>(
                   regionLoads_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -10120,7 +10120,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> coprocessorsBuilder_;
 
       /**
@@ -10322,7 +10322,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (coprocessorsBuilder_ == null) {
           ensureCoprocessorsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, coprocessors_);
           onChanged();
         } else {
@@ -10462,11 +10462,11 @@ public final class ClusterStatusProtos {
            getCoprocessorsBuilderList() {
         return getCoprocessorsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
           getCoprocessorsFieldBuilder() {
         if (coprocessorsBuilder_ == null) {
-          coprocessorsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          coprocessorsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder>(
                   coprocessors_,
                   ((bitField0_ & 0x00000020) == 0x00000020),
@@ -10658,7 +10658,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder> replLoadSourceBuilder_;
 
       /**
@@ -10840,7 +10840,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (replLoadSourceBuilder_ == null) {
           ensureReplLoadSourceIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, replLoadSource_);
           onChanged();
         } else {
@@ -10964,11 +10964,11 @@ public final class ClusterStatusProtos {
            getReplLoadSourceBuilderList() {
         return getReplLoadSourceFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder> 
           getReplLoadSourceFieldBuilder() {
         if (replLoadSourceBuilder_ == null) {
-          replLoadSourceBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          replLoadSourceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSource.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSourceOrBuilder>(
                   replLoadSource_,
                   ((bitField0_ & 0x00000200) == 0x00000200),
@@ -10980,7 +10980,7 @@ public final class ClusterStatusProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink replLoadSink_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder> replLoadSinkBuilder_;
       /**
        * 
@@ -11128,11 +11128,11 @@ public final class ClusterStatusProtos {
        *
        * optional .hbase.pb.ReplicationLoadSink replLoadSink = 11;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder> 
           getReplLoadSinkFieldBuilder() {
         if (replLoadSinkBuilder_ == null) {
-          replLoadSinkBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          replLoadSinkBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSink.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ReplicationLoadSinkOrBuilder>(
                   getReplLoadSink(),
                   getParentForChildren(),
@@ -11142,12 +11142,12 @@ public final class ClusterStatusProtos {
         return replLoadSinkBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11165,22 +11165,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ServerLoad parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ServerLoad(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11192,7 +11192,7 @@ public final class ClusterStatusProtos {
 
   public interface LiveServerInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.LiveServerInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ServerName server = 1;
@@ -11224,29 +11224,29 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.LiveServerInfo}
    */
   public  static final class LiveServerInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.LiveServerInfo)
       LiveServerInfoOrBuilder {
     // Use LiveServerInfo.newBuilder() to construct.
-    private LiveServerInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private LiveServerInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private LiveServerInfo() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private LiveServerInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11290,22 +11290,22 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_LiveServerInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_LiveServerInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11381,7 +11381,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServer());
@@ -11398,11 +11398,11 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServer());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getServerLoad());
       }
       size += unknownFields.getSerializedSize();
@@ -11457,61 +11457,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11529,7 +11529,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11537,15 +11537,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.LiveServerInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.LiveServerInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_LiveServerInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_LiveServerInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11558,12 +11558,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerFieldBuilder();
           getServerLoadFieldBuilder();
@@ -11586,7 +11586,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_LiveServerInfo_descriptor;
       }
@@ -11632,29 +11632,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo)other);
         } else {
@@ -11693,13 +11693,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11712,7 +11712,7 @@ public final class ClusterStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
        * required .hbase.pb.ServerName server = 1;
@@ -11815,11 +11815,11 @@ public final class ClusterStatusProtos {
       /**
        * required .hbase.pb.ServerName server = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerFieldBuilder() {
         if (serverBuilder_ == null) {
-          serverBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServer(),
                   getParentForChildren(),
@@ -11830,7 +11830,7 @@ public final class ClusterStatusProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad serverLoad_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder> serverLoadBuilder_;
       /**
        * required .hbase.pb.ServerLoad server_load = 2;
@@ -11933,11 +11933,11 @@ public final class ClusterStatusProtos {
       /**
        * required .hbase.pb.ServerLoad server_load = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder> 
           getServerLoadFieldBuilder() {
         if (serverLoadBuilder_ == null) {
-          serverLoadBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverLoadBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder>(
                   getServerLoad(),
                   getParentForChildren(),
@@ -11947,12 +11947,12 @@ public final class ClusterStatusProtos {
         return serverLoadBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11970,22 +11970,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public LiveServerInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new LiveServerInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11997,7 +11997,7 @@ public final class ClusterStatusProtos {
 
   public interface ClusterStatusOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ClusterStatus)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.HBaseVersionFileContent hbase_version = 1;
@@ -12171,11 +12171,11 @@ public final class ClusterStatusProtos {
    * Protobuf type {@code hbase.pb.ClusterStatus}
    */
   public  static final class ClusterStatus extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ClusterStatus)
       ClusterStatusOrBuilder {
     // Use ClusterStatus.newBuilder() to construct.
-    private ClusterStatus(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ClusterStatus(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ClusterStatus() {
@@ -12188,18 +12188,18 @@ public final class ClusterStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ClusterStatus(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12306,10 +12306,10 @@ public final class ClusterStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -12331,12 +12331,12 @@ public final class ClusterStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ClusterStatus_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ClusterStatus_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12655,7 +12655,7 @@ public final class ClusterStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getHbaseVersion());
@@ -12693,39 +12693,39 @@ public final class ClusterStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getHbaseVersion());
       }
       for (int i = 0; i < liveServers_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, liveServers_.get(i));
       }
       for (int i = 0; i < deadServers_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, deadServers_.get(i));
       }
       for (int i = 0; i < regionsInTransition_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, regionsInTransition_.get(i));
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getClusterId());
       }
       for (int i = 0; i < masterCoprocessors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, masterCoprocessors_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(7, getMaster());
       }
       for (int i = 0; i < backupMasters_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(8, backupMasters_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(9, balancerOn_);
       }
       size += unknownFields.getSerializedSize();
@@ -12820,7 +12820,7 @@ public final class ClusterStatusProtos {
       }
       if (hasBalancerOn()) {
         hash = (37 * hash) + BALANCER_ON_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getBalancerOn());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -12829,61 +12829,61 @@ public final class ClusterStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12901,7 +12901,7 @@ public final class ClusterStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12909,15 +12909,15 @@ public final class ClusterStatusProtos {
      * Protobuf type {@code hbase.pb.ClusterStatus}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ClusterStatus)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatusOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ClusterStatus_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ClusterStatus_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12930,12 +12930,12 @@ public final class ClusterStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getHbaseVersionFieldBuilder();
           getLiveServersFieldBuilder();
@@ -13002,7 +13002,7 @@ public final class ClusterStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.internal_static_hbase_pb_ClusterStatus_descriptor;
       }
@@ -13105,29 +13105,29 @@ public final class ClusterStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus)other);
         } else {
@@ -13160,7 +13160,7 @@ public final class ClusterStatusProtos {
               liveServers_ = other.liveServers_;
               bitField0_ = (bitField0_ & ~0x00000002);
               liveServersBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getLiveServersFieldBuilder() : null;
             } else {
               liveServersBuilder_.addAllMessages(other.liveServers_);
@@ -13186,7 +13186,7 @@ public final class ClusterStatusProtos {
               deadServers_ = other.deadServers_;
               bitField0_ = (bitField0_ & ~0x00000004);
               deadServersBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getDeadServersFieldBuilder() : null;
             } else {
               deadServersBuilder_.addAllMessages(other.deadServers_);
@@ -13212,7 +13212,7 @@ public final class ClusterStatusProtos {
               regionsInTransition_ = other.regionsInTransition_;
               bitField0_ = (bitField0_ & ~0x00000008);
               regionsInTransitionBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionsInTransitionFieldBuilder() : null;
             } else {
               regionsInTransitionBuilder_.addAllMessages(other.regionsInTransition_);
@@ -13241,7 +13241,7 @@ public final class ClusterStatusProtos {
               masterCoprocessors_ = other.masterCoprocessors_;
               bitField0_ = (bitField0_ & ~0x00000020);
               masterCoprocessorsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getMasterCoprocessorsFieldBuilder() : null;
             } else {
               masterCoprocessorsBuilder_.addAllMessages(other.masterCoprocessors_);
@@ -13270,7 +13270,7 @@ public final class ClusterStatusProtos {
               backupMasters_ = other.backupMasters_;
               bitField0_ = (bitField0_ & ~0x00000080);
               backupMastersBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getBackupMastersFieldBuilder() : null;
             } else {
               backupMastersBuilder_.addAllMessages(other.backupMasters_);
@@ -13330,13 +13330,13 @@ public final class ClusterStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13349,7 +13349,7 @@ public final class ClusterStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent hbaseVersion_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContentOrBuilder> hbaseVersionBuilder_;
       /**
        * optional .hbase.pb.HBaseVersionFileContent hbase_version = 1;
@@ -13452,11 +13452,11 @@ public final class ClusterStatusProtos {
       /**
        * optional .hbase.pb.HBaseVersionFileContent hbase_version = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContentOrBuilder> 
           getHbaseVersionFieldBuilder() {
         if (hbaseVersionBuilder_ == null) {
-          hbaseVersionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          hbaseVersionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContentOrBuilder>(
                   getHbaseVersion(),
                   getParentForChildren(),
@@ -13475,7 +13475,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfoOrBuilder> liveServersBuilder_;
 
       /**
@@ -13607,7 +13607,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (liveServersBuilder_ == null) {
           ensureLiveServersIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, liveServers_);
           onChanged();
         } else {
@@ -13691,11 +13691,11 @@ public final class ClusterStatusProtos {
            getLiveServersBuilderList() {
         return getLiveServersFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfoOrBuilder> 
           getLiveServersFieldBuilder() {
         if (liveServersBuilder_ == null) {
-          liveServersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          liveServersBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfoOrBuilder>(
                   liveServers_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -13715,7 +13715,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> deadServersBuilder_;
 
       /**
@@ -13847,7 +13847,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (deadServersBuilder_ == null) {
           ensureDeadServersIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, deadServers_);
           onChanged();
         } else {
@@ -13931,11 +13931,11 @@ public final class ClusterStatusProtos {
            getDeadServersBuilderList() {
         return getDeadServersFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getDeadServersFieldBuilder() {
         if (deadServersBuilder_ == null) {
-          deadServersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          deadServersBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   deadServers_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -13955,7 +13955,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransitionOrBuilder> regionsInTransitionBuilder_;
 
       /**
@@ -14087,7 +14087,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (regionsInTransitionBuilder_ == null) {
           ensureRegionsInTransitionIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionsInTransition_);
           onChanged();
         } else {
@@ -14171,11 +14171,11 @@ public final class ClusterStatusProtos {
            getRegionsInTransitionBuilderList() {
         return getRegionsInTransitionFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransitionOrBuilder> 
           getRegionsInTransitionFieldBuilder() {
         if (regionsInTransitionBuilder_ == null) {
-          regionsInTransitionBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionsInTransitionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransitionOrBuilder>(
                   regionsInTransition_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -14187,7 +14187,7 @@ public final class ClusterStatusProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId clusterId_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterIdOrBuilder> clusterIdBuilder_;
       /**
        * optional .hbase.pb.ClusterId cluster_id = 5;
@@ -14290,11 +14290,11 @@ public final class ClusterStatusProtos {
       /**
        * optional .hbase.pb.ClusterId cluster_id = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterIdOrBuilder> 
           getClusterIdFieldBuilder() {
         if (clusterIdBuilder_ == null) {
-          clusterIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          clusterIdBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.ClusterIdOrBuilder>(
                   getClusterId(),
                   getParentForChildren(),
@@ -14313,7 +14313,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> masterCoprocessorsBuilder_;
 
       /**
@@ -14445,7 +14445,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (masterCoprocessorsBuilder_ == null) {
           ensureMasterCoprocessorsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, masterCoprocessors_);
           onChanged();
         } else {
@@ -14529,11 +14529,11 @@ public final class ClusterStatusProtos {
            getMasterCoprocessorsBuilderList() {
         return getMasterCoprocessorsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder> 
           getMasterCoprocessorsFieldBuilder() {
         if (masterCoprocessorsBuilder_ == null) {
-          masterCoprocessorsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          masterCoprocessorsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder>(
                   masterCoprocessors_,
                   ((bitField0_ & 0x00000020) == 0x00000020),
@@ -14545,7 +14545,7 @@ public final class ClusterStatusProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName master_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> masterBuilder_;
       /**
        * optional .hbase.pb.ServerName master = 7;
@@ -14648,11 +14648,11 @@ public final class ClusterStatusProtos {
       /**
        * optional .hbase.pb.ServerName master = 7;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getMasterFieldBuilder() {
         if (masterBuilder_ == null) {
-          masterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          masterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getMaster(),
                   getParentForChildren(),
@@ -14671,7 +14671,7 @@ public final class ClusterStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> backupMastersBuilder_;
 
       /**
@@ -14803,7 +14803,7 @@ public final class ClusterStatusProtos {
           java.lang.Iterable values) {
         if (backupMastersBuilder_ == null) {
           ensureBackupMastersIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, backupMasters_);
           onChanged();
         } else {
@@ -14887,11 +14887,11 @@ public final class ClusterStatusProtos {
            getBackupMastersBuilderList() {
         return getBackupMastersFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getBackupMastersFieldBuilder() {
         if (backupMastersBuilder_ == null) {
-          backupMastersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          backupMastersBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   backupMasters_,
                   ((bitField0_ & 0x00000080) == 0x00000080),
@@ -14934,12 +14934,12 @@ public final class ClusterStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14957,22 +14957,22 @@ public final class ClusterStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ClusterStatus parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ClusterStatus(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14982,62 +14982,62 @@ public final class ClusterStatusProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionState_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionInTransition_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionInTransition_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StoreSequenceId_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StoreSequenceId_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionStoreSequenceIds_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionStoreSequenceIds_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionLoad_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionLoad_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationLoadSink_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationLoadSource_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicationLoadSource_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ServerLoad_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ServerLoad_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_LiveServerInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_LiveServerInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ClusterStatus_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ClusterStatus_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -15107,17 +15107,17 @@ public final class ClusterStatusProtos {
       "p.hbase.shaded.protobuf.generatedB\023Clust" +
       "erStatusProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.getDescriptor(),
@@ -15125,61 +15125,61 @@ public final class ClusterStatusProtos {
     internal_static_hbase_pb_RegionState_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_RegionState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionState_descriptor,
         new java.lang.String[] { "RegionInfo", "State", "Stamp", });
     internal_static_hbase_pb_RegionInTransition_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_RegionInTransition_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionInTransition_descriptor,
         new java.lang.String[] { "Spec", "RegionState", });
     internal_static_hbase_pb_StoreSequenceId_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_StoreSequenceId_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StoreSequenceId_descriptor,
         new java.lang.String[] { "FamilyName", "SequenceId", });
     internal_static_hbase_pb_RegionStoreSequenceIds_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_RegionStoreSequenceIds_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionStoreSequenceIds_descriptor,
         new java.lang.String[] { "LastFlushedSequenceId", "StoreSequenceId", });
     internal_static_hbase_pb_RegionLoad_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_RegionLoad_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionLoad_descriptor,
         new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", "FilteredReadRequestsCount", });
     internal_static_hbase_pb_ReplicationLoadSink_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicationLoadSink_descriptor,
         new java.lang.String[] { "AgeOfLastAppliedOp", "TimeStampsOfLastAppliedOp", });
     internal_static_hbase_pb_ReplicationLoadSource_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_ReplicationLoadSource_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicationLoadSource_descriptor,
         new java.lang.String[] { "PeerID", "AgeOfLastShippedOp", "SizeOfLogQueue", "TimeStampOfLastShippedOp", "ReplicationLag", });
     internal_static_hbase_pb_ServerLoad_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_ServerLoad_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ServerLoad_descriptor,
         new java.lang.String[] { "NumberOfRequests", "TotalNumberOfRequests", "UsedHeapMB", "MaxHeapMB", "RegionLoads", "Coprocessors", "ReportStartTime", "ReportEndTime", "InfoServerPort", "ReplLoadSource", "ReplLoadSink", });
     internal_static_hbase_pb_LiveServerInfo_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_LiveServerInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_LiveServerInfo_descriptor,
         new java.lang.String[] { "Server", "ServerLoad", });
     internal_static_hbase_pb_ClusterStatus_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_ClusterStatus_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ClusterStatus_descriptor,
         new java.lang.String[] { "HbaseVersion", "LiveServers", "DeadServers", "RegionsInTransition", "ClusterId", "MasterCoprocessors", "Master", "BackupMasters", "BalancerOn", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
index e1c605d..1c6c373 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ComparatorProtos {
   private ComparatorProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface ComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Comparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string name = 1;
@@ -29,7 +29,7 @@ public final class ComparatorProtos {
     /**
      * required string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
 
     /**
@@ -39,37 +39,37 @@ public final class ComparatorProtos {
     /**
      * optional bytes serialized_comparator = 2;
      */
-    com.google.protobuf.ByteString getSerializedComparator();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSerializedComparator();
   }
   /**
    * Protobuf type {@code hbase.pb.Comparator}
    */
   public  static final class Comparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Comparator)
       ComparatorOrBuilder {
     // Use Comparator.newBuilder() to construct.
-    private Comparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Comparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Comparator() {
       name_ = "";
-      serializedComparator_ = com.google.protobuf.ByteString.EMPTY;
+      serializedComparator_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Comparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -86,7 +86,7 @@ public final class ComparatorProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
@@ -98,22 +98,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_Comparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_Comparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -137,8 +137,8 @@ public final class ComparatorProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -149,22 +149,22 @@ public final class ComparatorProtos {
     /**
      * required string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int SERIALIZED_COMPARATOR_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString serializedComparator_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString serializedComparator_;
     /**
      * optional bytes serialized_comparator = 2;
      */
@@ -174,7 +174,7 @@ public final class ComparatorProtos {
     /**
      * optional bytes serialized_comparator = 2;
      */
-    public com.google.protobuf.ByteString getSerializedComparator() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSerializedComparator() {
       return serializedComparator_;
     }
 
@@ -192,10 +192,10 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBytes(2, serializedComparator_);
@@ -209,10 +209,10 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, serializedComparator_);
       }
       size += unknownFields.getSerializedSize();
@@ -267,61 +267,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -339,7 +339,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -347,15 +347,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.Comparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Comparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_Comparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_Comparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -368,12 +368,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -381,12 +381,12 @@ public final class ComparatorProtos {
         super.clear();
         name_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
-        serializedComparator_ = com.google.protobuf.ByteString.EMPTY;
+        serializedComparator_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_Comparator_descriptor;
       }
@@ -424,29 +424,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator)other);
         } else {
@@ -478,13 +478,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -509,8 +509,8 @@ public final class ComparatorProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -523,17 +523,17 @@ public final class ComparatorProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -562,7 +562,7 @@ public final class ComparatorProtos {
        * required string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -572,7 +572,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString serializedComparator_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString serializedComparator_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes serialized_comparator = 2;
        */
@@ -582,13 +582,13 @@ public final class ComparatorProtos {
       /**
        * optional bytes serialized_comparator = 2;
        */
-      public com.google.protobuf.ByteString getSerializedComparator() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSerializedComparator() {
         return serializedComparator_;
       }
       /**
        * optional bytes serialized_comparator = 2;
        */
-      public Builder setSerializedComparator(com.google.protobuf.ByteString value) {
+      public Builder setSerializedComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -607,12 +607,12 @@ public final class ComparatorProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -630,22 +630,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Comparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Comparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -657,7 +657,7 @@ public final class ComparatorProtos {
 
   public interface ByteArrayComparableOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ByteArrayComparable)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes value = 1;
@@ -666,36 +666,36 @@ public final class ComparatorProtos {
     /**
      * optional bytes value = 1;
      */
-    com.google.protobuf.ByteString getValue();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
   }
   /**
    * Protobuf type {@code hbase.pb.ByteArrayComparable}
    */
   public  static final class ByteArrayComparable extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ByteArrayComparable)
       ByteArrayComparableOrBuilder {
     // Use ByteArrayComparable.newBuilder() to construct.
-    private ByteArrayComparable(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ByteArrayComparable(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ByteArrayComparable() {
-      value_ = com.google.protobuf.ByteString.EMPTY;
+      value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ByteArrayComparable(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -718,22 +718,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_ByteArrayComparable_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_ByteArrayComparable_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -742,7 +742,7 @@ public final class ComparatorProtos {
 
     private int bitField0_;
     public static final int VALUE_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString value_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_;
     /**
      * optional bytes value = 1;
      */
@@ -752,7 +752,7 @@ public final class ComparatorProtos {
     /**
      * optional bytes value = 1;
      */
-    public com.google.protobuf.ByteString getValue() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
@@ -766,7 +766,7 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, value_);
@@ -780,7 +780,7 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, value_);
       }
       size += unknownFields.getSerializedSize();
@@ -826,61 +826,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -898,7 +898,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -906,15 +906,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.ByteArrayComparable}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ByteArrayComparable)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_ByteArrayComparable_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_ByteArrayComparable_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -927,23 +927,23 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_ByteArrayComparable_descriptor;
       }
@@ -977,29 +977,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable)other);
         } else {
@@ -1023,13 +1023,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1041,7 +1041,7 @@ public final class ComparatorProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes value = 1;
        */
@@ -1051,13 +1051,13 @@ public final class ComparatorProtos {
       /**
        * optional bytes value = 1;
        */
-      public com.google.protobuf.ByteString getValue() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
         return value_;
       }
       /**
        * optional bytes value = 1;
        */
-      public Builder setValue(com.google.protobuf.ByteString value) {
+      public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1076,12 +1076,12 @@ public final class ComparatorProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1099,22 +1099,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ByteArrayComparable parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ByteArrayComparable(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1126,7 +1126,7 @@ public final class ComparatorProtos {
 
   public interface BinaryComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BinaryComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -1145,29 +1145,29 @@ public final class ComparatorProtos {
    * Protobuf type {@code hbase.pb.BinaryComparator}
    */
   public  static final class BinaryComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BinaryComparator)
       BinaryComparatorOrBuilder {
     // Use BinaryComparator.newBuilder() to construct.
-    private BinaryComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BinaryComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BinaryComparator() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BinaryComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1198,22 +1198,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1256,7 +1256,7 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getComparable());
@@ -1270,7 +1270,7 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getComparable());
       }
       size += unknownFields.getSerializedSize();
@@ -1316,61 +1316,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1388,7 +1388,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1396,15 +1396,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.BinaryComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BinaryComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1417,12 +1417,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparableFieldBuilder();
         }
@@ -1438,7 +1438,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryComparator_descriptor;
       }
@@ -1476,29 +1476,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator)other);
         } else {
@@ -1525,13 +1525,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1544,7 +1544,7 @@ public final class ComparatorProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable comparable_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> comparableBuilder_;
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -1647,11 +1647,11 @@ public final class ComparatorProtos {
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> 
           getComparableFieldBuilder() {
         if (comparableBuilder_ == null) {
-          comparableBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparableBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder>(
                   getComparable(),
                   getParentForChildren(),
@@ -1661,12 +1661,12 @@ public final class ComparatorProtos {
         return comparableBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1684,22 +1684,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BinaryComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BinaryComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1711,7 +1711,7 @@ public final class ComparatorProtos {
 
   public interface LongComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.LongComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -1730,29 +1730,29 @@ public final class ComparatorProtos {
    * Protobuf type {@code hbase.pb.LongComparator}
    */
   public  static final class LongComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.LongComparator)
       LongComparatorOrBuilder {
     // Use LongComparator.newBuilder() to construct.
-    private LongComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private LongComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private LongComparator() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private LongComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1783,22 +1783,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_LongComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_LongComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1841,7 +1841,7 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getComparable());
@@ -1855,7 +1855,7 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getComparable());
       }
       size += unknownFields.getSerializedSize();
@@ -1901,61 +1901,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1973,7 +1973,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1981,15 +1981,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.LongComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.LongComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_LongComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_LongComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2002,12 +2002,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparableFieldBuilder();
         }
@@ -2023,7 +2023,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_LongComparator_descriptor;
       }
@@ -2061,29 +2061,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator)other);
         } else {
@@ -2110,13 +2110,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.LongComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2129,7 +2129,7 @@ public final class ComparatorProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable comparable_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> comparableBuilder_;
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -2232,11 +2232,11 @@ public final class ComparatorProtos {
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> 
           getComparableFieldBuilder() {
         if (comparableBuilder_ == null) {
-          comparableBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparableBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder>(
                   getComparable(),
                   getParentForChildren(),
@@ -2246,12 +2246,12 @@ public final class ComparatorProtos {
         return comparableBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2269,22 +2269,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public LongComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new LongComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2296,7 +2296,7 @@ public final class ComparatorProtos {
 
   public interface BinaryPrefixComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BinaryPrefixComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -2315,29 +2315,29 @@ public final class ComparatorProtos {
    * Protobuf type {@code hbase.pb.BinaryPrefixComparator}
    */
   public  static final class BinaryPrefixComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BinaryPrefixComparator)
       BinaryPrefixComparatorOrBuilder {
     // Use BinaryPrefixComparator.newBuilder() to construct.
-    private BinaryPrefixComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BinaryPrefixComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BinaryPrefixComparator() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BinaryPrefixComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2368,22 +2368,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryPrefixComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryPrefixComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2426,7 +2426,7 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getComparable());
@@ -2440,7 +2440,7 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getComparable());
       }
       size += unknownFields.getSerializedSize();
@@ -2486,61 +2486,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2558,7 +2558,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2566,15 +2566,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.BinaryPrefixComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BinaryPrefixComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryPrefixComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryPrefixComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2587,12 +2587,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparableFieldBuilder();
         }
@@ -2608,7 +2608,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BinaryPrefixComparator_descriptor;
       }
@@ -2646,29 +2646,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator)other);
         } else {
@@ -2695,13 +2695,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BinaryPrefixComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2714,7 +2714,7 @@ public final class ComparatorProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable comparable_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> comparableBuilder_;
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -2817,11 +2817,11 @@ public final class ComparatorProtos {
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> 
           getComparableFieldBuilder() {
         if (comparableBuilder_ == null) {
-          comparableBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparableBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder>(
                   getComparable(),
                   getParentForChildren(),
@@ -2831,12 +2831,12 @@ public final class ComparatorProtos {
         return comparableBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2854,22 +2854,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BinaryPrefixComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BinaryPrefixComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2881,7 +2881,7 @@ public final class ComparatorProtos {
 
   public interface BitComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BitComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -2909,11 +2909,11 @@ public final class ComparatorProtos {
    * Protobuf type {@code hbase.pb.BitComparator}
    */
   public  static final class BitComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BitComparator)
       BitComparatorOrBuilder {
     // Use BitComparator.newBuilder() to construct.
-    private BitComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BitComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BitComparator() {
@@ -2921,18 +2921,18 @@ public final class ComparatorProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BitComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2974,22 +2974,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BitComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BitComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3000,7 +3000,7 @@ public final class ComparatorProtos {
      * Protobuf enum {@code hbase.pb.BitComparator.BitwiseOp}
      */
     public enum BitwiseOp
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * AND = 1;
        */
@@ -3050,27 +3050,27 @@ public final class ComparatorProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           BitwiseOp> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public BitwiseOp findValueByNumber(int number) {
                 return BitwiseOp.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator.getDescriptor().getEnumTypes().get(0);
       }
@@ -3078,7 +3078,7 @@ public final class ComparatorProtos {
       private static final BitwiseOp[] VALUES = values();
 
       public static BitwiseOp valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -3151,7 +3151,7 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getComparable());
@@ -3168,11 +3168,11 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getComparable());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, bitwiseOp_);
       }
       size += unknownFields.getSerializedSize();
@@ -3226,61 +3226,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3298,7 +3298,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3306,15 +3306,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.BitComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BitComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BitComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BitComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3327,12 +3327,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparableFieldBuilder();
         }
@@ -3350,7 +3350,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_BitComparator_descriptor;
       }
@@ -3392,29 +3392,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator)other);
         } else {
@@ -3447,13 +3447,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.BitComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3466,7 +3466,7 @@ public final class ComparatorProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable comparable_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> comparableBuilder_;
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
@@ -3569,11 +3569,11 @@ public final class ComparatorProtos {
       /**
        * required .hbase.pb.ByteArrayComparable comparable = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> 
           getComparableFieldBuilder() {
         if (comparableBuilder_ == null) {
-          comparableBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparableBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder>(
                   getComparable(),
                   getParentForChildren(),
@@ -3619,12 +3619,12 @@ public final class ComparatorProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3642,22 +3642,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BitComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BitComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3669,34 +3669,34 @@ public final class ComparatorProtos {
 
   public interface NullComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NullComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.NullComparator}
    */
   public  static final class NullComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NullComparator)
       NullComparatorOrBuilder {
     // Use NullComparator.newBuilder() to construct.
-    private NullComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NullComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NullComparator() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NullComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3714,22 +3714,22 @@ public final class ComparatorProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_NullComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_NullComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3746,7 +3746,7 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -3790,61 +3790,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3862,7 +3862,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3870,15 +3870,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.NullComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NullComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_NullComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_NullComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3891,12 +3891,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3905,7 +3905,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_NullComparator_descriptor;
       }
@@ -3932,29 +3932,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator)other);
         } else {
@@ -3975,13 +3975,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.NullComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3992,12 +3992,12 @@ public final class ComparatorProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4015,22 +4015,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NullComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NullComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4042,7 +4042,7 @@ public final class ComparatorProtos {
 
   public interface RegexStringComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegexStringComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string pattern = 1;
@@ -4055,7 +4055,7 @@ public final class ComparatorProtos {
     /**
      * required string pattern = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getPatternBytes();
 
     /**
@@ -4078,7 +4078,7 @@ public final class ComparatorProtos {
     /**
      * required string charset = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCharsetBytes();
 
     /**
@@ -4092,18 +4092,18 @@ public final class ComparatorProtos {
     /**
      * optional string engine = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getEngineBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.RegexStringComparator}
    */
   public  static final class RegexStringComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegexStringComparator)
       RegexStringComparatorOrBuilder {
     // Use RegexStringComparator.newBuilder() to construct.
-    private RegexStringComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegexStringComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegexStringComparator() {
@@ -4114,18 +4114,18 @@ public final class ComparatorProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegexStringComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4142,7 +4142,7 @@ public final class ComparatorProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               pattern_ = bs;
               break;
@@ -4153,35 +4153,35 @@ public final class ComparatorProtos {
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               charset_ = bs;
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               engine_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_RegexStringComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_RegexStringComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4205,8 +4205,8 @@ public final class ComparatorProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           pattern_ = s;
@@ -4217,17 +4217,17 @@ public final class ComparatorProtos {
     /**
      * required string pattern = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getPatternBytes() {
       java.lang.Object ref = pattern_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         pattern_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -4262,8 +4262,8 @@ public final class ComparatorProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           charset_ = s;
@@ -4274,17 +4274,17 @@ public final class ComparatorProtos {
     /**
      * required string charset = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCharsetBytes() {
       java.lang.Object ref = charset_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         charset_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -4304,8 +4304,8 @@ public final class ComparatorProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           engine_ = s;
@@ -4316,17 +4316,17 @@ public final class ComparatorProtos {
     /**
      * optional string engine = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getEngineBytes() {
       java.lang.Object ref = engine_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         engine_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -4352,19 +4352,19 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, pattern_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, pattern_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt32(2, patternFlags_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, charset_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, charset_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, engine_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, engine_);
       }
       unknownFields.writeTo(output);
     }
@@ -4375,17 +4375,17 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, pattern_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, pattern_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, patternFlags_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, charset_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, charset_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, engine_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, engine_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -4457,61 +4457,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4529,7 +4529,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4537,15 +4537,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.RegexStringComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegexStringComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_RegexStringComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_RegexStringComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4558,12 +4558,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -4580,7 +4580,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_RegexStringComparator_descriptor;
       }
@@ -4626,29 +4626,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator)other);
         } else {
@@ -4696,13 +4696,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.RegexStringComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4727,8 +4727,8 @@ public final class ComparatorProtos {
       public java.lang.String getPattern() {
         java.lang.Object ref = pattern_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             pattern_ = s;
@@ -4741,17 +4741,17 @@ public final class ComparatorProtos {
       /**
        * required string pattern = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getPatternBytes() {
         java.lang.Object ref = pattern_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           pattern_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -4780,7 +4780,7 @@ public final class ComparatorProtos {
        * required string pattern = 1;
        */
       public Builder setPatternBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4835,8 +4835,8 @@ public final class ComparatorProtos {
       public java.lang.String getCharset() {
         java.lang.Object ref = charset_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             charset_ = s;
@@ -4849,17 +4849,17 @@ public final class ComparatorProtos {
       /**
        * required string charset = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getCharsetBytes() {
         java.lang.Object ref = charset_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           charset_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -4888,7 +4888,7 @@ public final class ComparatorProtos {
        * required string charset = 3;
        */
       public Builder setCharsetBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4911,8 +4911,8 @@ public final class ComparatorProtos {
       public java.lang.String getEngine() {
         java.lang.Object ref = engine_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             engine_ = s;
@@ -4925,17 +4925,17 @@ public final class ComparatorProtos {
       /**
        * optional string engine = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getEngineBytes() {
         java.lang.Object ref = engine_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           engine_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -4964,7 +4964,7 @@ public final class ComparatorProtos {
        * optional string engine = 4;
        */
       public Builder setEngineBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4974,12 +4974,12 @@ public final class ComparatorProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4997,22 +4997,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegexStringComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegexStringComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5024,7 +5024,7 @@ public final class ComparatorProtos {
 
   public interface SubstringComparatorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SubstringComparator)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string substr = 1;
@@ -5037,18 +5037,18 @@ public final class ComparatorProtos {
     /**
      * required string substr = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSubstrBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.SubstringComparator}
    */
   public  static final class SubstringComparator extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SubstringComparator)
       SubstringComparatorOrBuilder {
     // Use SubstringComparator.newBuilder() to construct.
-    private SubstringComparator(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SubstringComparator(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SubstringComparator() {
@@ -5056,18 +5056,18 @@ public final class ComparatorProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SubstringComparator(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5084,29 +5084,29 @@ public final class ComparatorProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               substr_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_SubstringComparator_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_SubstringComparator_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5130,8 +5130,8 @@ public final class ComparatorProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           substr_ = s;
@@ -5142,17 +5142,17 @@ public final class ComparatorProtos {
     /**
      * required string substr = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSubstrBytes() {
       java.lang.Object ref = substr_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         substr_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -5170,10 +5170,10 @@ public final class ComparatorProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, substr_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, substr_);
       }
       unknownFields.writeTo(output);
     }
@@ -5184,7 +5184,7 @@ public final class ComparatorProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, substr_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, substr_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -5229,61 +5229,61 @@ public final class ComparatorProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5301,7 +5301,7 @@ public final class ComparatorProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5309,15 +5309,15 @@ public final class ComparatorProtos {
      * Protobuf type {@code hbase.pb.SubstringComparator}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SubstringComparator)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparatorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_SubstringComparator_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_SubstringComparator_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5330,12 +5330,12 @@ public final class ComparatorProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -5346,7 +5346,7 @@ public final class ComparatorProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.internal_static_hbase_pb_SubstringComparator_descriptor;
       }
@@ -5380,29 +5380,29 @@ public final class ComparatorProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator)other);
         } else {
@@ -5431,13 +5431,13 @@ public final class ComparatorProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.SubstringComparator) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5462,8 +5462,8 @@ public final class ComparatorProtos {
       public java.lang.String getSubstr() {
         java.lang.Object ref = substr_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             substr_ = s;
@@ -5476,17 +5476,17 @@ public final class ComparatorProtos {
       /**
        * required string substr = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getSubstrBytes() {
         java.lang.Object ref = substr_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           substr_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -5515,7 +5515,7 @@ public final class ComparatorProtos {
        * required string substr = 1;
        */
       public Builder setSubstrBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5525,12 +5525,12 @@ public final class ComparatorProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5548,22 +5548,22 @@ public final class ComparatorProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SubstringComparator parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SubstringComparator(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5573,57 +5573,57 @@ public final class ComparatorProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Comparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Comparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ByteArrayComparable_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ByteArrayComparable_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BinaryComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BinaryComparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_LongComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_LongComparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BinaryPrefixComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BinaryPrefixComparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BitComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BitComparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NullComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NullComparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegexStringComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegexStringComparator_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SubstringComparator_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SubstringComparator_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -5647,70 +5647,70 @@ public final class ComparatorProtos {
       "p.hbase.shaded.protobuf.generatedB\020Compa" +
       "ratorProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_Comparator_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_Comparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Comparator_descriptor,
         new java.lang.String[] { "Name", "SerializedComparator", });
     internal_static_hbase_pb_ByteArrayComparable_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_ByteArrayComparable_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ByteArrayComparable_descriptor,
         new java.lang.String[] { "Value", });
     internal_static_hbase_pb_BinaryComparator_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_BinaryComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BinaryComparator_descriptor,
         new java.lang.String[] { "Comparable", });
     internal_static_hbase_pb_LongComparator_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_LongComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_LongComparator_descriptor,
         new java.lang.String[] { "Comparable", });
     internal_static_hbase_pb_BinaryPrefixComparator_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_BinaryPrefixComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BinaryPrefixComparator_descriptor,
         new java.lang.String[] { "Comparable", });
     internal_static_hbase_pb_BitComparator_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_BitComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BitComparator_descriptor,
         new java.lang.String[] { "Comparable", "BitwiseOp", });
     internal_static_hbase_pb_NullComparator_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_NullComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NullComparator_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_RegexStringComparator_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_RegexStringComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegexStringComparator_descriptor,
         new java.lang.String[] { "Pattern", "PatternFlags", "Charset", "Engine", });
     internal_static_hbase_pb_SubstringComparator_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_SubstringComparator_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SubstringComparator_descriptor,
         new java.lang.String[] { "Substr", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
index 689d33f..b195b5a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class EncryptionProtos {
   private EncryptionProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface WrappedKeyOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WrappedKey)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string algorithm = 1;
@@ -29,7 +29,7 @@ public final class EncryptionProtos {
     /**
      * required string algorithm = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getAlgorithmBytes();
 
     /**
@@ -48,7 +48,7 @@ public final class EncryptionProtos {
     /**
      * required bytes data = 3;
      */
-    com.google.protobuf.ByteString getData();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getData();
 
     /**
      * optional bytes iv = 4;
@@ -57,7 +57,7 @@ public final class EncryptionProtos {
     /**
      * optional bytes iv = 4;
      */
-    com.google.protobuf.ByteString getIv();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getIv();
 
     /**
      * optional bytes hash = 5;
@@ -66,40 +66,40 @@ public final class EncryptionProtos {
     /**
      * optional bytes hash = 5;
      */
-    com.google.protobuf.ByteString getHash();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getHash();
   }
   /**
    * Protobuf type {@code hbase.pb.WrappedKey}
    */
   public  static final class WrappedKey extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WrappedKey)
       WrappedKeyOrBuilder {
     // Use WrappedKey.newBuilder() to construct.
-    private WrappedKey(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WrappedKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WrappedKey() {
       algorithm_ = "";
       length_ = 0;
-      data_ = com.google.protobuf.ByteString.EMPTY;
-      iv_ = com.google.protobuf.ByteString.EMPTY;
-      hash_ = com.google.protobuf.ByteString.EMPTY;
+      data_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      iv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      hash_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WrappedKey(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -116,7 +116,7 @@ public final class EncryptionProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               algorithm_ = bs;
               break;
@@ -143,22 +143,22 @@ public final class EncryptionProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.internal_static_hbase_pb_WrappedKey_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.internal_static_hbase_pb_WrappedKey_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -182,8 +182,8 @@ public final class EncryptionProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           algorithm_ = s;
@@ -194,17 +194,17 @@ public final class EncryptionProtos {
     /**
      * required string algorithm = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getAlgorithmBytes() {
       java.lang.Object ref = algorithm_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         algorithm_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -224,7 +224,7 @@ public final class EncryptionProtos {
     }
 
     public static final int DATA_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString data_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data_;
     /**
      * required bytes data = 3;
      */
@@ -234,12 +234,12 @@ public final class EncryptionProtos {
     /**
      * required bytes data = 3;
      */
-    public com.google.protobuf.ByteString getData() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getData() {
       return data_;
     }
 
     public static final int IV_FIELD_NUMBER = 4;
-    private com.google.protobuf.ByteString iv_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString iv_;
     /**
      * optional bytes iv = 4;
      */
@@ -249,12 +249,12 @@ public final class EncryptionProtos {
     /**
      * optional bytes iv = 4;
      */
-    public com.google.protobuf.ByteString getIv() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getIv() {
       return iv_;
     }
 
     public static final int HASH_FIELD_NUMBER = 5;
-    private com.google.protobuf.ByteString hash_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString hash_;
     /**
      * optional bytes hash = 5;
      */
@@ -264,7 +264,7 @@ public final class EncryptionProtos {
     /**
      * optional bytes hash = 5;
      */
-    public com.google.protobuf.ByteString getHash() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getHash() {
       return hash_;
     }
 
@@ -290,10 +290,10 @@ public final class EncryptionProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, algorithm_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, algorithm_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt32(2, length_);
@@ -316,22 +316,22 @@ public final class EncryptionProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, algorithm_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, algorithm_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, length_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, data_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(4, iv_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(5, hash_);
       }
       size += unknownFields.getSerializedSize();
@@ -413,61 +413,61 @@ public final class EncryptionProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -485,7 +485,7 @@ public final class EncryptionProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -493,15 +493,15 @@ public final class EncryptionProtos {
      * Protobuf type {@code hbase.pb.WrappedKey}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WrappedKey)
         org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKeyOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.internal_static_hbase_pb_WrappedKey_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.internal_static_hbase_pb_WrappedKey_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -514,12 +514,12 @@ public final class EncryptionProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -529,16 +529,16 @@ public final class EncryptionProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         length_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
-        data_ = com.google.protobuf.ByteString.EMPTY;
+        data_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
-        iv_ = com.google.protobuf.ByteString.EMPTY;
+        iv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
-        hash_ = com.google.protobuf.ByteString.EMPTY;
+        hash_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.internal_static_hbase_pb_WrappedKey_descriptor;
       }
@@ -588,29 +588,29 @@ public final class EncryptionProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey)other);
         } else {
@@ -657,13 +657,13 @@ public final class EncryptionProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos.WrappedKey) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -688,8 +688,8 @@ public final class EncryptionProtos {
       public java.lang.String getAlgorithm() {
         java.lang.Object ref = algorithm_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             algorithm_ = s;
@@ -702,17 +702,17 @@ public final class EncryptionProtos {
       /**
        * required string algorithm = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getAlgorithmBytes() {
         java.lang.Object ref = algorithm_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           algorithm_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -741,7 +741,7 @@ public final class EncryptionProtos {
        * required string algorithm = 1;
        */
       public Builder setAlgorithmBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -783,7 +783,7 @@ public final class EncryptionProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString data_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes data = 3;
        */
@@ -793,13 +793,13 @@ public final class EncryptionProtos {
       /**
        * required bytes data = 3;
        */
-      public com.google.protobuf.ByteString getData() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getData() {
         return data_;
       }
       /**
        * required bytes data = 3;
        */
-      public Builder setData(com.google.protobuf.ByteString value) {
+      public Builder setData(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -818,7 +818,7 @@ public final class EncryptionProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString iv_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString iv_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes iv = 4;
        */
@@ -828,13 +828,13 @@ public final class EncryptionProtos {
       /**
        * optional bytes iv = 4;
        */
-      public com.google.protobuf.ByteString getIv() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getIv() {
         return iv_;
       }
       /**
        * optional bytes iv = 4;
        */
-      public Builder setIv(com.google.protobuf.ByteString value) {
+      public Builder setIv(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -853,7 +853,7 @@ public final class EncryptionProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString hash_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString hash_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes hash = 5;
        */
@@ -863,13 +863,13 @@ public final class EncryptionProtos {
       /**
        * optional bytes hash = 5;
        */
-      public com.google.protobuf.ByteString getHash() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getHash() {
         return hash_;
       }
       /**
        * optional bytes hash = 5;
        */
-      public Builder setHash(com.google.protobuf.ByteString value) {
+      public Builder setHash(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -888,12 +888,12 @@ public final class EncryptionProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -911,22 +911,22 @@ public final class EncryptionProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WrappedKey parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WrappedKey(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -936,17 +936,17 @@ public final class EncryptionProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WrappedKey_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WrappedKey_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -956,22 +956,22 @@ public final class EncryptionProtos {
       "1org.apache.hadoop.hbase.shaded.protobuf" +
       ".generatedB\020EncryptionProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_WrappedKey_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_WrappedKey_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WrappedKey_descriptor,
         new java.lang.String[] { "Algorithm", "Length", "Data", "Iv", "Hash", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
index d696e90..ccb7e86 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ErrorHandlingProtos {
   private ErrorHandlingProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface StackTraceElementMessageOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StackTraceElementMessage)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional string declaring_class = 1;
@@ -29,7 +29,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional string declaring_class = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getDeclaringClassBytes();
 
     /**
@@ -43,7 +43,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional string method_name = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMethodNameBytes();
 
     /**
@@ -57,7 +57,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional string file_name = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getFileNameBytes();
 
     /**
@@ -79,11 +79,11 @@ public final class ErrorHandlingProtos {
    * Protobuf type {@code hbase.pb.StackTraceElementMessage}
    */
   public  static final class StackTraceElementMessage extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StackTraceElementMessage)
       StackTraceElementMessageOrBuilder {
     // Use StackTraceElementMessage.newBuilder() to construct.
-    private StackTraceElementMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StackTraceElementMessage(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StackTraceElementMessage() {
@@ -94,18 +94,18 @@ public final class ErrorHandlingProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StackTraceElementMessage(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -122,19 +122,19 @@ public final class ErrorHandlingProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               declaringClass_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               methodName_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               fileName_ = bs;
               break;
@@ -146,22 +146,22 @@ public final class ErrorHandlingProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_StackTraceElementMessage_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_StackTraceElementMessage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -185,8 +185,8 @@ public final class ErrorHandlingProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           declaringClass_ = s;
@@ -197,17 +197,17 @@ public final class ErrorHandlingProtos {
     /**
      * optional string declaring_class = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getDeclaringClassBytes() {
       java.lang.Object ref = declaringClass_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         declaringClass_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -227,8 +227,8 @@ public final class ErrorHandlingProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           methodName_ = s;
@@ -239,17 +239,17 @@ public final class ErrorHandlingProtos {
     /**
      * optional string method_name = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMethodNameBytes() {
       java.lang.Object ref = methodName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         methodName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -269,8 +269,8 @@ public final class ErrorHandlingProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           fileName_ = s;
@@ -281,17 +281,17 @@ public final class ErrorHandlingProtos {
     /**
      * optional string file_name = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getFileNameBytes() {
       java.lang.Object ref = fileName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         fileName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -320,16 +320,16 @@ public final class ErrorHandlingProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, declaringClass_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, declaringClass_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, methodName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, methodName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, fileName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, fileName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeInt32(4, lineNumber_);
@@ -343,16 +343,16 @@ public final class ErrorHandlingProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, declaringClass_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, declaringClass_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, methodName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, methodName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, fileName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, fileName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(4, lineNumber_);
       }
       size += unknownFields.getSerializedSize();
@@ -425,61 +425,61 @@ public final class ErrorHandlingProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -497,7 +497,7 @@ public final class ErrorHandlingProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -511,15 +511,15 @@ public final class ErrorHandlingProtos {
      * Protobuf type {@code hbase.pb.StackTraceElementMessage}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StackTraceElementMessage)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessageOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_StackTraceElementMessage_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_StackTraceElementMessage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -532,12 +532,12 @@ public final class ErrorHandlingProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -554,7 +554,7 @@ public final class ErrorHandlingProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_StackTraceElementMessage_descriptor;
       }
@@ -600,29 +600,29 @@ public final class ErrorHandlingProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage)other);
         } else {
@@ -661,13 +661,13 @@ public final class ErrorHandlingProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -692,8 +692,8 @@ public final class ErrorHandlingProtos {
       public java.lang.String getDeclaringClass() {
         java.lang.Object ref = declaringClass_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             declaringClass_ = s;
@@ -706,17 +706,17 @@ public final class ErrorHandlingProtos {
       /**
        * optional string declaring_class = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getDeclaringClassBytes() {
         java.lang.Object ref = declaringClass_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           declaringClass_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -745,7 +745,7 @@ public final class ErrorHandlingProtos {
        * optional string declaring_class = 1;
        */
       public Builder setDeclaringClassBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -768,8 +768,8 @@ public final class ErrorHandlingProtos {
       public java.lang.String getMethodName() {
         java.lang.Object ref = methodName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             methodName_ = s;
@@ -782,17 +782,17 @@ public final class ErrorHandlingProtos {
       /**
        * optional string method_name = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getMethodNameBytes() {
         java.lang.Object ref = methodName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           methodName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -821,7 +821,7 @@ public final class ErrorHandlingProtos {
        * optional string method_name = 2;
        */
       public Builder setMethodNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -844,8 +844,8 @@ public final class ErrorHandlingProtos {
       public java.lang.String getFileName() {
         java.lang.Object ref = fileName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             fileName_ = s;
@@ -858,17 +858,17 @@ public final class ErrorHandlingProtos {
       /**
        * optional string file_name = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getFileNameBytes() {
         java.lang.Object ref = fileName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           fileName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -897,7 +897,7 @@ public final class ErrorHandlingProtos {
        * optional string file_name = 3;
        */
       public Builder setFileNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -939,12 +939,12 @@ public final class ErrorHandlingProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -962,22 +962,22 @@ public final class ErrorHandlingProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StackTraceElementMessage parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StackTraceElementMessage(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -989,7 +989,7 @@ public final class ErrorHandlingProtos {
 
   public interface GenericExceptionMessageOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GenericExceptionMessage)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional string class_name = 1;
@@ -1002,7 +1002,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional string class_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClassNameBytes();
 
     /**
@@ -1016,7 +1016,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional string message = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMessageBytes();
 
     /**
@@ -1026,7 +1026,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional bytes error_info = 3;
      */
-    com.google.protobuf.ByteString getErrorInfo();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getErrorInfo();
 
     /**
      * repeated .hbase.pb.StackTraceElementMessage trace = 4;
@@ -1064,33 +1064,33 @@ public final class ErrorHandlingProtos {
    * Protobuf type {@code hbase.pb.GenericExceptionMessage}
    */
   public  static final class GenericExceptionMessage extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GenericExceptionMessage)
       GenericExceptionMessageOrBuilder {
     // Use GenericExceptionMessage.newBuilder() to construct.
-    private GenericExceptionMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GenericExceptionMessage(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GenericExceptionMessage() {
       className_ = "";
       message_ = "";
-      errorInfo_ = com.google.protobuf.ByteString.EMPTY;
+      errorInfo_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       trace_ = java.util.Collections.emptyList();
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GenericExceptionMessage(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1107,13 +1107,13 @@ public final class ErrorHandlingProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               className_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               message_ = bs;
               break;
@@ -1134,10 +1134,10 @@ public final class ErrorHandlingProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
@@ -1147,12 +1147,12 @@ public final class ErrorHandlingProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_GenericExceptionMessage_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_GenericExceptionMessage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1176,8 +1176,8 @@ public final class ErrorHandlingProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           className_ = s;
@@ -1188,17 +1188,17 @@ public final class ErrorHandlingProtos {
     /**
      * optional string class_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClassNameBytes() {
       java.lang.Object ref = className_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         className_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -1218,8 +1218,8 @@ public final class ErrorHandlingProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           message_ = s;
@@ -1230,22 +1230,22 @@ public final class ErrorHandlingProtos {
     /**
      * optional string message = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMessageBytes() {
       java.lang.Object ref = message_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         message_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int ERROR_INFO_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString errorInfo_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString errorInfo_;
     /**
      * optional bytes error_info = 3;
      */
@@ -1255,7 +1255,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional bytes error_info = 3;
      */
-    public com.google.protobuf.ByteString getErrorInfo() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getErrorInfo() {
       return errorInfo_;
     }
 
@@ -1304,13 +1304,13 @@ public final class ErrorHandlingProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, className_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, className_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, message_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, message_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeBytes(3, errorInfo_);
@@ -1327,17 +1327,17 @@ public final class ErrorHandlingProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, className_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, className_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, message_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, message_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, errorInfo_);
       }
       for (int i = 0; i < trace_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, trace_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -1407,61 +1407,61 @@ public final class ErrorHandlingProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1479,7 +1479,7 @@ public final class ErrorHandlingProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1495,15 +1495,15 @@ public final class ErrorHandlingProtos {
      * Protobuf type {@code hbase.pb.GenericExceptionMessage}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GenericExceptionMessage)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessageOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_GenericExceptionMessage_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_GenericExceptionMessage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1516,12 +1516,12 @@ public final class ErrorHandlingProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTraceFieldBuilder();
         }
@@ -1532,7 +1532,7 @@ public final class ErrorHandlingProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         message_ = "";
         bitField0_ = (bitField0_ & ~0x00000002);
-        errorInfo_ = com.google.protobuf.ByteString.EMPTY;
+        errorInfo_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         if (traceBuilder_ == null) {
           trace_ = java.util.Collections.emptyList();
@@ -1543,7 +1543,7 @@ public final class ErrorHandlingProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_GenericExceptionMessage_descriptor;
       }
@@ -1594,29 +1594,29 @@ public final class ErrorHandlingProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage)other);
         } else {
@@ -1659,7 +1659,7 @@ public final class ErrorHandlingProtos {
               trace_ = other.trace_;
               bitField0_ = (bitField0_ & ~0x00000008);
               traceBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTraceFieldBuilder() : null;
             } else {
               traceBuilder_.addAllMessages(other.trace_);
@@ -1676,13 +1676,13 @@ public final class ErrorHandlingProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1707,8 +1707,8 @@ public final class ErrorHandlingProtos {
       public java.lang.String getClassName() {
         java.lang.Object ref = className_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             className_ = s;
@@ -1721,17 +1721,17 @@ public final class ErrorHandlingProtos {
       /**
        * optional string class_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getClassNameBytes() {
         java.lang.Object ref = className_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           className_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1760,7 +1760,7 @@ public final class ErrorHandlingProtos {
        * optional string class_name = 1;
        */
       public Builder setClassNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1783,8 +1783,8 @@ public final class ErrorHandlingProtos {
       public java.lang.String getMessage() {
         java.lang.Object ref = message_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             message_ = s;
@@ -1797,17 +1797,17 @@ public final class ErrorHandlingProtos {
       /**
        * optional string message = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getMessageBytes() {
         java.lang.Object ref = message_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           message_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1836,7 +1836,7 @@ public final class ErrorHandlingProtos {
        * optional string message = 2;
        */
       public Builder setMessageBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1846,7 +1846,7 @@ public final class ErrorHandlingProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString errorInfo_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString errorInfo_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes error_info = 3;
        */
@@ -1856,13 +1856,13 @@ public final class ErrorHandlingProtos {
       /**
        * optional bytes error_info = 3;
        */
-      public com.google.protobuf.ByteString getErrorInfo() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getErrorInfo() {
         return errorInfo_;
       }
       /**
        * optional bytes error_info = 3;
        */
-      public Builder setErrorInfo(com.google.protobuf.ByteString value) {
+      public Builder setErrorInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1890,7 +1890,7 @@ public final class ErrorHandlingProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessageOrBuilder> traceBuilder_;
 
       /**
@@ -2022,7 +2022,7 @@ public final class ErrorHandlingProtos {
           java.lang.Iterable values) {
         if (traceBuilder_ == null) {
           ensureTraceIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, trace_);
           onChanged();
         } else {
@@ -2106,11 +2106,11 @@ public final class ErrorHandlingProtos {
            getTraceBuilderList() {
         return getTraceFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessageOrBuilder> 
           getTraceFieldBuilder() {
         if (traceBuilder_ == null) {
-          traceBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          traceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessageOrBuilder>(
                   trace_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -2121,12 +2121,12 @@ public final class ErrorHandlingProtos {
         return traceBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2144,22 +2144,22 @@ public final class ErrorHandlingProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GenericExceptionMessage parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GenericExceptionMessage(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2171,7 +2171,7 @@ public final class ErrorHandlingProtos {
 
   public interface ForeignExceptionMessageOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ForeignExceptionMessage)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional string source = 1;
@@ -2184,7 +2184,7 @@ public final class ErrorHandlingProtos {
     /**
      * optional string source = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSourceBytes();
 
     /**
@@ -2210,11 +2210,11 @@ public final class ErrorHandlingProtos {
    * Protobuf type {@code hbase.pb.ForeignExceptionMessage}
    */
   public  static final class ForeignExceptionMessage extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ForeignExceptionMessage)
       ForeignExceptionMessageOrBuilder {
     // Use ForeignExceptionMessage.newBuilder() to construct.
-    private ForeignExceptionMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ForeignExceptionMessage(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ForeignExceptionMessage() {
@@ -2222,18 +2222,18 @@ public final class ErrorHandlingProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ForeignExceptionMessage(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2250,7 +2250,7 @@ public final class ErrorHandlingProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               source_ = bs;
               break;
@@ -2270,22 +2270,22 @@ public final class ErrorHandlingProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_ForeignExceptionMessage_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_ForeignExceptionMessage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2309,8 +2309,8 @@ public final class ErrorHandlingProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           source_ = s;
@@ -2321,17 +2321,17 @@ public final class ErrorHandlingProtos {
     /**
      * optional string source = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSourceBytes() {
       java.lang.Object ref = source_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         source_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -2366,10 +2366,10 @@ public final class ErrorHandlingProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, source_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, source_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeMessage(2, getGenericException());
@@ -2383,10 +2383,10 @@ public final class ErrorHandlingProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, source_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, source_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getGenericException());
       }
       size += unknownFields.getSerializedSize();
@@ -2441,61 +2441,61 @@ public final class ErrorHandlingProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2513,7 +2513,7 @@ public final class ErrorHandlingProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2527,15 +2527,15 @@ public final class ErrorHandlingProtos {
      * Protobuf type {@code hbase.pb.ForeignExceptionMessage}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ForeignExceptionMessage)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_ForeignExceptionMessage_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_ForeignExceptionMessage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2548,12 +2548,12 @@ public final class ErrorHandlingProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getGenericExceptionFieldBuilder();
         }
@@ -2571,7 +2571,7 @@ public final class ErrorHandlingProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.internal_static_hbase_pb_ForeignExceptionMessage_descriptor;
       }
@@ -2613,29 +2613,29 @@ public final class ErrorHandlingProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage)other);
         } else {
@@ -2664,13 +2664,13 @@ public final class ErrorHandlingProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2695,8 +2695,8 @@ public final class ErrorHandlingProtos {
       public java.lang.String getSource() {
         java.lang.Object ref = source_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             source_ = s;
@@ -2709,17 +2709,17 @@ public final class ErrorHandlingProtos {
       /**
        * optional string source = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getSourceBytes() {
         java.lang.Object ref = source_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           source_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -2748,7 +2748,7 @@ public final class ErrorHandlingProtos {
        * optional string source = 1;
        */
       public Builder setSourceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2759,7 +2759,7 @@ public final class ErrorHandlingProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage genericException_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessageOrBuilder> genericExceptionBuilder_;
       /**
        * optional .hbase.pb.GenericExceptionMessage generic_exception = 2;
@@ -2862,11 +2862,11 @@ public final class ErrorHandlingProtos {
       /**
        * optional .hbase.pb.GenericExceptionMessage generic_exception = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessageOrBuilder> 
           getGenericExceptionFieldBuilder() {
         if (genericExceptionBuilder_ == null) {
-          genericExceptionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          genericExceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessageOrBuilder>(
                   getGenericException(),
                   getParentForChildren(),
@@ -2876,12 +2876,12 @@ public final class ErrorHandlingProtos {
         return genericExceptionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2899,22 +2899,22 @@ public final class ErrorHandlingProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ForeignExceptionMessage parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ForeignExceptionMessage(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2924,27 +2924,27 @@ public final class ErrorHandlingProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StackTraceElementMessage_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StackTraceElementMessage_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GenericExceptionMessage_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GenericExceptionMessage_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ForeignExceptionMessage_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ForeignExceptionMessage_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -2961,34 +2961,34 @@ public final class ErrorHandlingProtos {
       ".hadoop.hbase.shaded.protobuf.generatedB" +
       "\023ErrorHandlingProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_StackTraceElementMessage_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_StackTraceElementMessage_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StackTraceElementMessage_descriptor,
         new java.lang.String[] { "DeclaringClass", "MethodName", "FileName", "LineNumber", });
     internal_static_hbase_pb_GenericExceptionMessage_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_GenericExceptionMessage_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GenericExceptionMessage_descriptor,
         new java.lang.String[] { "ClassName", "Message", "ErrorInfo", "Trace", });
     internal_static_hbase_pb_ForeignExceptionMessage_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_ForeignExceptionMessage_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ForeignExceptionMessage_descriptor,
         new java.lang.String[] { "Source", "GenericException", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
index 5b456c8..cbe92ca 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class FSProtos {
   private FSProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface HBaseVersionFileContentOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.HBaseVersionFileContent)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string version = 1;
@@ -29,7 +29,7 @@ public final class FSProtos {
     /**
      * required string version = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getVersionBytes();
   }
   /**
@@ -41,11 +41,11 @@ public final class FSProtos {
    * Protobuf type {@code hbase.pb.HBaseVersionFileContent}
    */
   public  static final class HBaseVersionFileContent extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.HBaseVersionFileContent)
       HBaseVersionFileContentOrBuilder {
     // Use HBaseVersionFileContent.newBuilder() to construct.
-    private HBaseVersionFileContent(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private HBaseVersionFileContent(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private HBaseVersionFileContent() {
@@ -53,18 +53,18 @@ public final class FSProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private HBaseVersionFileContent(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -81,29 +81,29 @@ public final class FSProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               version_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_HBaseVersionFileContent_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_HBaseVersionFileContent_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -127,8 +127,8 @@ public final class FSProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           version_ = s;
@@ -139,17 +139,17 @@ public final class FSProtos {
     /**
      * required string version = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getVersionBytes() {
       java.lang.Object ref = version_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         version_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -167,10 +167,10 @@ public final class FSProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, version_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, version_);
       }
       unknownFields.writeTo(output);
     }
@@ -181,7 +181,7 @@ public final class FSProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, version_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, version_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -226,61 +226,61 @@ public final class FSProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -298,7 +298,7 @@ public final class FSProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -311,15 +311,15 @@ public final class FSProtos {
      * Protobuf type {@code hbase.pb.HBaseVersionFileContent}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.HBaseVersionFileContent)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContentOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_HBaseVersionFileContent_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_HBaseVersionFileContent_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -332,12 +332,12 @@ public final class FSProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -348,7 +348,7 @@ public final class FSProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_HBaseVersionFileContent_descriptor;
       }
@@ -382,29 +382,29 @@ public final class FSProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent)other);
         } else {
@@ -433,13 +433,13 @@ public final class FSProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -464,8 +464,8 @@ public final class FSProtos {
       public java.lang.String getVersion() {
         java.lang.Object ref = version_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             version_ = s;
@@ -478,17 +478,17 @@ public final class FSProtos {
       /**
        * required string version = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getVersionBytes() {
         java.lang.Object ref = version_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           version_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -517,7 +517,7 @@ public final class FSProtos {
        * required string version = 1;
        */
       public Builder setVersionBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -527,12 +527,12 @@ public final class FSProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -550,22 +550,22 @@ public final class FSProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public HBaseVersionFileContent parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new HBaseVersionFileContent(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -577,7 +577,7 @@ public final class FSProtos {
 
   public interface ReferenceOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Reference)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes splitkey = 1;
@@ -586,7 +586,7 @@ public final class FSProtos {
     /**
      * required bytes splitkey = 1;
      */
-    com.google.protobuf.ByteString getSplitkey();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitkey();
 
     /**
      * required .hbase.pb.Reference.Range range = 2;
@@ -606,31 +606,31 @@ public final class FSProtos {
    * Protobuf type {@code hbase.pb.Reference}
    */
   public  static final class Reference extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Reference)
       ReferenceOrBuilder {
     // Use Reference.newBuilder() to construct.
-    private Reference(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Reference(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Reference() {
-      splitkey_ = com.google.protobuf.ByteString.EMPTY;
+      splitkey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       range_ = 0;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Reference(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -664,22 +664,22 @@ public final class FSProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_Reference_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_Reference_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -690,7 +690,7 @@ public final class FSProtos {
      * Protobuf enum {@code hbase.pb.Reference.Range}
      */
     public enum Range
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * TOP = 0;
        */
@@ -731,27 +731,27 @@ public final class FSProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Range> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Range findValueByNumber(int number) {
                 return Range.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference.getDescriptor().getEnumTypes().get(0);
       }
@@ -759,7 +759,7 @@ public final class FSProtos {
       private static final Range[] VALUES = values();
 
       public static Range valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -778,7 +778,7 @@ public final class FSProtos {
 
     private int bitField0_;
     public static final int SPLITKEY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString splitkey_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString splitkey_;
     /**
      * required bytes splitkey = 1;
      */
@@ -788,7 +788,7 @@ public final class FSProtos {
     /**
      * required bytes splitkey = 1;
      */
-    public com.google.protobuf.ByteString getSplitkey() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitkey() {
       return splitkey_;
     }
 
@@ -826,7 +826,7 @@ public final class FSProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, splitkey_);
@@ -843,11 +843,11 @@ public final class FSProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, splitkey_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, range_);
       }
       size += unknownFields.getSerializedSize();
@@ -901,61 +901,61 @@ public final class FSProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -973,7 +973,7 @@ public final class FSProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -986,15 +986,15 @@ public final class FSProtos {
      * Protobuf type {@code hbase.pb.Reference}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Reference)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.ReferenceOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_Reference_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_Reference_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1007,25 +1007,25 @@ public final class FSProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        splitkey_ = com.google.protobuf.ByteString.EMPTY;
+        splitkey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         range_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.internal_static_hbase_pb_Reference_descriptor;
       }
@@ -1063,29 +1063,29 @@ public final class FSProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference)other);
         } else {
@@ -1118,13 +1118,13 @@ public final class FSProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1136,7 +1136,7 @@ public final class FSProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString splitkey_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString splitkey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes splitkey = 1;
        */
@@ -1146,13 +1146,13 @@ public final class FSProtos {
       /**
        * required bytes splitkey = 1;
        */
-      public com.google.protobuf.ByteString getSplitkey() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitkey() {
         return splitkey_;
       }
       /**
        * required bytes splitkey = 1;
        */
-      public Builder setSplitkey(com.google.protobuf.ByteString value) {
+      public Builder setSplitkey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1207,12 +1207,12 @@ public final class FSProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1230,22 +1230,22 @@ public final class FSProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Reference parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Reference(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1255,22 +1255,22 @@ public final class FSProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_HBaseVersionFileContent_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_HBaseVersionFileContent_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Reference_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Reference_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -1281,28 +1281,28 @@ public final class FSProtos {
       "OTTOM\020\001BB\n1org.apache.hadoop.hbase.shade" +
       "d.protobuf.generatedB\010FSProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_HBaseVersionFileContent_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_HBaseVersionFileContent_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_HBaseVersionFileContent_descriptor,
         new java.lang.String[] { "Version", });
     internal_static_hbase_pb_Reference_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_Reference_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Reference_descriptor,
         new java.lang.String[] { "Splitkey", "Range", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
index 7204813..b329e45 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class FilterProtos {
   private FilterProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface FilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Filter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string name = 1;
@@ -29,7 +29,7 @@ public final class FilterProtos {
     /**
      * required string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
 
     /**
@@ -39,37 +39,37 @@ public final class FilterProtos {
     /**
      * optional bytes serialized_filter = 2;
      */
-    com.google.protobuf.ByteString getSerializedFilter();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSerializedFilter();
   }
   /**
    * Protobuf type {@code hbase.pb.Filter}
    */
   public  static final class Filter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Filter)
       FilterOrBuilder {
     // Use Filter.newBuilder() to construct.
-    private Filter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Filter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Filter() {
       name_ = "";
-      serializedFilter_ = com.google.protobuf.ByteString.EMPTY;
+      serializedFilter_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Filter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -86,7 +86,7 @@ public final class FilterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
@@ -98,22 +98,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_Filter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_Filter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -137,8 +137,8 @@ public final class FilterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -149,22 +149,22 @@ public final class FilterProtos {
     /**
      * required string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int SERIALIZED_FILTER_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString serializedFilter_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString serializedFilter_;
     /**
      * optional bytes serialized_filter = 2;
      */
@@ -174,7 +174,7 @@ public final class FilterProtos {
     /**
      * optional bytes serialized_filter = 2;
      */
-    public com.google.protobuf.ByteString getSerializedFilter() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSerializedFilter() {
       return serializedFilter_;
     }
 
@@ -192,10 +192,10 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBytes(2, serializedFilter_);
@@ -209,10 +209,10 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, serializedFilter_);
       }
       size += unknownFields.getSerializedSize();
@@ -267,61 +267,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -339,7 +339,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -347,15 +347,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.Filter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Filter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_Filter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_Filter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -368,12 +368,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -381,12 +381,12 @@ public final class FilterProtos {
         super.clear();
         name_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
-        serializedFilter_ = com.google.protobuf.ByteString.EMPTY;
+        serializedFilter_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_Filter_descriptor;
       }
@@ -424,29 +424,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter)other);
         } else {
@@ -478,13 +478,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -509,8 +509,8 @@ public final class FilterProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -523,17 +523,17 @@ public final class FilterProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -562,7 +562,7 @@ public final class FilterProtos {
        * required string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -572,7 +572,7 @@ public final class FilterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString serializedFilter_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString serializedFilter_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes serialized_filter = 2;
        */
@@ -582,13 +582,13 @@ public final class FilterProtos {
       /**
        * optional bytes serialized_filter = 2;
        */
-      public com.google.protobuf.ByteString getSerializedFilter() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSerializedFilter() {
         return serializedFilter_;
       }
       /**
        * optional bytes serialized_filter = 2;
        */
-      public Builder setSerializedFilter(com.google.protobuf.ByteString value) {
+      public Builder setSerializedFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -607,12 +607,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -630,22 +630,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Filter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Filter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -657,7 +657,7 @@ public final class FilterProtos {
 
   public interface ColumnCountGetFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ColumnCountGetFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int32 limit = 1;
@@ -672,11 +672,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.ColumnCountGetFilter}
    */
   public  static final class ColumnCountGetFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ColumnCountGetFilter)
       ColumnCountGetFilterOrBuilder {
     // Use ColumnCountGetFilter.newBuilder() to construct.
-    private ColumnCountGetFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ColumnCountGetFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ColumnCountGetFilter() {
@@ -684,18 +684,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ColumnCountGetFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -718,22 +718,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnCountGetFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnCountGetFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -770,7 +770,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(1, limit_);
@@ -784,7 +784,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(1, limit_);
       }
       size += unknownFields.getSerializedSize();
@@ -830,61 +830,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -902,7 +902,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -910,15 +910,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.ColumnCountGetFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ColumnCountGetFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnCountGetFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnCountGetFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -931,12 +931,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -947,7 +947,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnCountGetFilter_descriptor;
       }
@@ -981,29 +981,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter)other);
         } else {
@@ -1030,13 +1030,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnCountGetFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1080,12 +1080,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1103,22 +1103,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ColumnCountGetFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ColumnCountGetFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1130,7 +1130,7 @@ public final class FilterProtos {
 
   public interface ColumnPaginationFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ColumnPaginationFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int32 limit = 1;
@@ -1157,38 +1157,38 @@ public final class FilterProtos {
     /**
      * optional bytes column_offset = 3;
      */
-    com.google.protobuf.ByteString getColumnOffset();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnOffset();
   }
   /**
    * Protobuf type {@code hbase.pb.ColumnPaginationFilter}
    */
   public  static final class ColumnPaginationFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ColumnPaginationFilter)
       ColumnPaginationFilterOrBuilder {
     // Use ColumnPaginationFilter.newBuilder() to construct.
-    private ColumnPaginationFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ColumnPaginationFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ColumnPaginationFilter() {
       limit_ = 0;
       offset_ = 0;
-      columnOffset_ = com.google.protobuf.ByteString.EMPTY;
+      columnOffset_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ColumnPaginationFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1221,22 +1221,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPaginationFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPaginationFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1275,7 +1275,7 @@ public final class FilterProtos {
     }
 
     public static final int COLUMN_OFFSET_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString columnOffset_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnOffset_;
     /**
      * optional bytes column_offset = 3;
      */
@@ -1285,7 +1285,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_offset = 3;
      */
-    public com.google.protobuf.ByteString getColumnOffset() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnOffset() {
       return columnOffset_;
     }
 
@@ -1303,7 +1303,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(1, limit_);
@@ -1323,15 +1323,15 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(1, limit_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, offset_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, columnOffset_);
       }
       size += unknownFields.getSerializedSize();
@@ -1395,61 +1395,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1467,7 +1467,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1475,15 +1475,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.ColumnPaginationFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ColumnPaginationFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPaginationFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPaginationFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1496,12 +1496,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -1511,12 +1511,12 @@ public final class FilterProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         offset_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
-        columnOffset_ = com.google.protobuf.ByteString.EMPTY;
+        columnOffset_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPaginationFilter_descriptor;
       }
@@ -1558,29 +1558,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter)other);
         } else {
@@ -1613,13 +1613,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPaginationFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1695,7 +1695,7 @@ public final class FilterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString columnOffset_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnOffset_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes column_offset = 3;
        */
@@ -1705,13 +1705,13 @@ public final class FilterProtos {
       /**
        * optional bytes column_offset = 3;
        */
-      public com.google.protobuf.ByteString getColumnOffset() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnOffset() {
         return columnOffset_;
       }
       /**
        * optional bytes column_offset = 3;
        */
-      public Builder setColumnOffset(com.google.protobuf.ByteString value) {
+      public Builder setColumnOffset(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1730,12 +1730,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1753,22 +1753,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ColumnPaginationFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ColumnPaginationFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1780,7 +1780,7 @@ public final class FilterProtos {
 
   public interface ColumnPrefixFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ColumnPrefixFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes prefix = 1;
@@ -1789,36 +1789,36 @@ public final class FilterProtos {
     /**
      * required bytes prefix = 1;
      */
-    com.google.protobuf.ByteString getPrefix();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPrefix();
   }
   /**
    * Protobuf type {@code hbase.pb.ColumnPrefixFilter}
    */
   public  static final class ColumnPrefixFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ColumnPrefixFilter)
       ColumnPrefixFilterOrBuilder {
     // Use ColumnPrefixFilter.newBuilder() to construct.
-    private ColumnPrefixFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ColumnPrefixFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ColumnPrefixFilter() {
-      prefix_ = com.google.protobuf.ByteString.EMPTY;
+      prefix_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ColumnPrefixFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1841,22 +1841,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPrefixFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPrefixFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1865,7 +1865,7 @@ public final class FilterProtos {
 
     private int bitField0_;
     public static final int PREFIX_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString prefix_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString prefix_;
     /**
      * required bytes prefix = 1;
      */
@@ -1875,7 +1875,7 @@ public final class FilterProtos {
     /**
      * required bytes prefix = 1;
      */
-    public com.google.protobuf.ByteString getPrefix() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPrefix() {
       return prefix_;
     }
 
@@ -1893,7 +1893,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, prefix_);
@@ -1907,7 +1907,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, prefix_);
       }
       size += unknownFields.getSerializedSize();
@@ -1953,61 +1953,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2025,7 +2025,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2033,15 +2033,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.ColumnPrefixFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ColumnPrefixFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPrefixFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPrefixFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2054,23 +2054,23 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        prefix_ = com.google.protobuf.ByteString.EMPTY;
+        prefix_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnPrefixFilter_descriptor;
       }
@@ -2104,29 +2104,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter)other);
         } else {
@@ -2153,13 +2153,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnPrefixFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2171,7 +2171,7 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString prefix_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString prefix_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes prefix = 1;
        */
@@ -2181,13 +2181,13 @@ public final class FilterProtos {
       /**
        * required bytes prefix = 1;
        */
-      public com.google.protobuf.ByteString getPrefix() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPrefix() {
         return prefix_;
       }
       /**
        * required bytes prefix = 1;
        */
-      public Builder setPrefix(com.google.protobuf.ByteString value) {
+      public Builder setPrefix(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2206,12 +2206,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2229,22 +2229,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ColumnPrefixFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ColumnPrefixFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2256,7 +2256,7 @@ public final class FilterProtos {
 
   public interface ColumnRangeFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ColumnRangeFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes min_column = 1;
@@ -2265,7 +2265,7 @@ public final class FilterProtos {
     /**
      * optional bytes min_column = 1;
      */
-    com.google.protobuf.ByteString getMinColumn();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMinColumn();
 
     /**
      * optional bool min_column_inclusive = 2;
@@ -2283,7 +2283,7 @@ public final class FilterProtos {
     /**
      * optional bytes max_column = 3;
      */
-    com.google.protobuf.ByteString getMaxColumn();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMaxColumn();
 
     /**
      * optional bool max_column_inclusive = 4;
@@ -2298,33 +2298,33 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.ColumnRangeFilter}
    */
   public  static final class ColumnRangeFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ColumnRangeFilter)
       ColumnRangeFilterOrBuilder {
     // Use ColumnRangeFilter.newBuilder() to construct.
-    private ColumnRangeFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ColumnRangeFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ColumnRangeFilter() {
-      minColumn_ = com.google.protobuf.ByteString.EMPTY;
+      minColumn_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       minColumnInclusive_ = false;
-      maxColumn_ = com.google.protobuf.ByteString.EMPTY;
+      maxColumn_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       maxColumnInclusive_ = false;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ColumnRangeFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2362,22 +2362,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnRangeFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnRangeFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2386,7 +2386,7 @@ public final class FilterProtos {
 
     private int bitField0_;
     public static final int MIN_COLUMN_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString minColumn_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString minColumn_;
     /**
      * optional bytes min_column = 1;
      */
@@ -2396,7 +2396,7 @@ public final class FilterProtos {
     /**
      * optional bytes min_column = 1;
      */
-    public com.google.protobuf.ByteString getMinColumn() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMinColumn() {
       return minColumn_;
     }
 
@@ -2416,7 +2416,7 @@ public final class FilterProtos {
     }
 
     public static final int MAX_COLUMN_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString maxColumn_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString maxColumn_;
     /**
      * optional bytes max_column = 3;
      */
@@ -2426,7 +2426,7 @@ public final class FilterProtos {
     /**
      * optional bytes max_column = 3;
      */
-    public com.google.protobuf.ByteString getMaxColumn() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMaxColumn() {
       return maxColumn_;
     }
 
@@ -2455,7 +2455,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, minColumn_);
@@ -2478,19 +2478,19 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, minColumn_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, minColumnInclusive_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, maxColumn_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, maxColumnInclusive_);
       }
       size += unknownFields.getSerializedSize();
@@ -2547,7 +2547,7 @@ public final class FilterProtos {
       }
       if (hasMinColumnInclusive()) {
         hash = (37 * hash) + MIN_COLUMN_INCLUSIVE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getMinColumnInclusive());
       }
       if (hasMaxColumn()) {
@@ -2556,7 +2556,7 @@ public final class FilterProtos {
       }
       if (hasMaxColumnInclusive()) {
         hash = (37 * hash) + MAX_COLUMN_INCLUSIVE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getMaxColumnInclusive());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -2565,61 +2565,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2637,7 +2637,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2645,15 +2645,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.ColumnRangeFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ColumnRangeFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnRangeFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnRangeFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2666,29 +2666,29 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        minColumn_ = com.google.protobuf.ByteString.EMPTY;
+        minColumn_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         minColumnInclusive_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
-        maxColumn_ = com.google.protobuf.ByteString.EMPTY;
+        maxColumn_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         maxColumnInclusive_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ColumnRangeFilter_descriptor;
       }
@@ -2734,29 +2734,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter)other);
         } else {
@@ -2789,13 +2789,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ColumnRangeFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2807,7 +2807,7 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString minColumn_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString minColumn_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes min_column = 1;
        */
@@ -2817,13 +2817,13 @@ public final class FilterProtos {
       /**
        * optional bytes min_column = 1;
        */
-      public com.google.protobuf.ByteString getMinColumn() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMinColumn() {
         return minColumn_;
       }
       /**
        * optional bytes min_column = 1;
        */
-      public Builder setMinColumn(com.google.protobuf.ByteString value) {
+      public Builder setMinColumn(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2874,7 +2874,7 @@ public final class FilterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString maxColumn_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString maxColumn_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes max_column = 3;
        */
@@ -2884,13 +2884,13 @@ public final class FilterProtos {
       /**
        * optional bytes max_column = 3;
        */
-      public com.google.protobuf.ByteString getMaxColumn() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getMaxColumn() {
         return maxColumn_;
       }
       /**
        * optional bytes max_column = 3;
        */
-      public Builder setMaxColumn(com.google.protobuf.ByteString value) {
+      public Builder setMaxColumn(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2941,12 +2941,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2964,22 +2964,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ColumnRangeFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ColumnRangeFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2991,7 +2991,7 @@ public final class FilterProtos {
 
   public interface CompareFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CompareFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.CompareType compare_op = 1;
@@ -3019,11 +3019,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.CompareFilter}
    */
   public  static final class CompareFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CompareFilter)
       CompareFilterOrBuilder {
     // Use CompareFilter.newBuilder() to construct.
-    private CompareFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CompareFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CompareFilter() {
@@ -3031,18 +3031,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CompareFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3084,22 +3084,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_CompareFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_CompareFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3164,7 +3164,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, compareOp_);
@@ -3181,11 +3181,11 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, compareOp_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getComparator());
       }
       size += unknownFields.getSerializedSize();
@@ -3239,61 +3239,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3311,7 +3311,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3319,15 +3319,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.CompareFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CompareFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_CompareFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_CompareFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3340,12 +3340,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparatorFieldBuilder();
         }
@@ -3363,7 +3363,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_CompareFilter_descriptor;
       }
@@ -3405,29 +3405,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter)other);
         } else {
@@ -3462,13 +3462,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3517,7 +3517,7 @@ public final class FilterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator comparator_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> comparatorBuilder_;
       /**
        * optional .hbase.pb.Comparator comparator = 2;
@@ -3620,11 +3620,11 @@ public final class FilterProtos {
       /**
        * optional .hbase.pb.Comparator comparator = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> 
           getComparatorFieldBuilder() {
         if (comparatorBuilder_ == null) {
-          comparatorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparatorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder>(
                   getComparator(),
                   getParentForChildren(),
@@ -3634,12 +3634,12 @@ public final class FilterProtos {
         return comparatorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3657,22 +3657,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CompareFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CompareFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3684,7 +3684,7 @@ public final class FilterProtos {
 
   public interface DependentColumnFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DependentColumnFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -3706,7 +3706,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_family = 2;
      */
-    com.google.protobuf.ByteString getColumnFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily();
 
     /**
      * optional bytes column_qualifier = 3;
@@ -3715,7 +3715,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_qualifier = 3;
      */
-    com.google.protobuf.ByteString getColumnQualifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnQualifier();
 
     /**
      * optional bool drop_dependent_column = 4;
@@ -3730,32 +3730,32 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.DependentColumnFilter}
    */
   public  static final class DependentColumnFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DependentColumnFilter)
       DependentColumnFilterOrBuilder {
     // Use DependentColumnFilter.newBuilder() to construct.
-    private DependentColumnFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DependentColumnFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DependentColumnFilter() {
-      columnFamily_ = com.google.protobuf.ByteString.EMPTY;
-      columnQualifier_ = com.google.protobuf.ByteString.EMPTY;
+      columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      columnQualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       dropDependentColumn_ = false;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DependentColumnFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3801,22 +3801,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_DependentColumnFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_DependentColumnFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3846,7 +3846,7 @@ public final class FilterProtos {
     }
 
     public static final int COLUMN_FAMILY_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString columnFamily_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnFamily_;
     /**
      * optional bytes column_family = 2;
      */
@@ -3856,12 +3856,12 @@ public final class FilterProtos {
     /**
      * optional bytes column_family = 2;
      */
-    public com.google.protobuf.ByteString getColumnFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily() {
       return columnFamily_;
     }
 
     public static final int COLUMN_QUALIFIER_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString columnQualifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnQualifier_;
     /**
      * optional bytes column_qualifier = 3;
      */
@@ -3871,7 +3871,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_qualifier = 3;
      */
-    public com.google.protobuf.ByteString getColumnQualifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnQualifier() {
       return columnQualifier_;
     }
 
@@ -3908,7 +3908,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getCompareFilter());
@@ -3931,19 +3931,19 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getCompareFilter());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, columnFamily_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, columnQualifier_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, dropDependentColumn_);
       }
       size += unknownFields.getSerializedSize();
@@ -4008,7 +4008,7 @@ public final class FilterProtos {
       }
       if (hasDropDependentColumn()) {
         hash = (37 * hash) + DROP_DEPENDENT_COLUMN_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDropDependentColumn());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -4017,61 +4017,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4089,7 +4089,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4097,15 +4097,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.DependentColumnFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DependentColumnFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_DependentColumnFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_DependentColumnFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4118,12 +4118,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getCompareFilterFieldBuilder();
         }
@@ -4136,16 +4136,16 @@ public final class FilterProtos {
           compareFilterBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+        columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        columnQualifier_ = com.google.protobuf.ByteString.EMPTY;
+        columnQualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         dropDependentColumn_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_DependentColumnFilter_descriptor;
       }
@@ -4195,29 +4195,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter)other);
         } else {
@@ -4256,13 +4256,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.DependentColumnFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4275,7 +4275,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter compareFilter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> compareFilterBuilder_;
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -4378,11 +4378,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> 
           getCompareFilterFieldBuilder() {
         if (compareFilterBuilder_ == null) {
-          compareFilterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          compareFilterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder>(
                   getCompareFilter(),
                   getParentForChildren(),
@@ -4392,7 +4392,7 @@ public final class FilterProtos {
         return compareFilterBuilder_;
       }
 
-      private com.google.protobuf.ByteString columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes column_family = 2;
        */
@@ -4402,13 +4402,13 @@ public final class FilterProtos {
       /**
        * optional bytes column_family = 2;
        */
-      public com.google.protobuf.ByteString getColumnFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily() {
         return columnFamily_;
       }
       /**
        * optional bytes column_family = 2;
        */
-      public Builder setColumnFamily(com.google.protobuf.ByteString value) {
+      public Builder setColumnFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4427,7 +4427,7 @@ public final class FilterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString columnQualifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnQualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes column_qualifier = 3;
        */
@@ -4437,13 +4437,13 @@ public final class FilterProtos {
       /**
        * optional bytes column_qualifier = 3;
        */
-      public com.google.protobuf.ByteString getColumnQualifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnQualifier() {
         return columnQualifier_;
       }
       /**
        * optional bytes column_qualifier = 3;
        */
-      public Builder setColumnQualifier(com.google.protobuf.ByteString value) {
+      public Builder setColumnQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4494,12 +4494,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4517,22 +4517,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DependentColumnFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DependentColumnFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4544,7 +4544,7 @@ public final class FilterProtos {
 
   public interface FamilyFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FamilyFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -4563,29 +4563,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.FamilyFilter}
    */
   public  static final class FamilyFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FamilyFilter)
       FamilyFilterOrBuilder {
     // Use FamilyFilter.newBuilder() to construct.
-    private FamilyFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FamilyFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FamilyFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FamilyFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4616,22 +4616,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FamilyFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FamilyFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4678,7 +4678,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getCompareFilter());
@@ -4692,7 +4692,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getCompareFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -4738,61 +4738,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4810,7 +4810,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4818,15 +4818,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FamilyFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FamilyFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FamilyFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FamilyFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4839,12 +4839,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getCompareFilterFieldBuilder();
         }
@@ -4860,7 +4860,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FamilyFilter_descriptor;
       }
@@ -4898,29 +4898,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter)other);
         } else {
@@ -4950,13 +4950,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FamilyFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4969,7 +4969,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter compareFilter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> compareFilterBuilder_;
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -5072,11 +5072,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> 
           getCompareFilterFieldBuilder() {
         if (compareFilterBuilder_ == null) {
-          compareFilterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          compareFilterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder>(
                   getCompareFilter(),
                   getParentForChildren(),
@@ -5086,12 +5086,12 @@ public final class FilterProtos {
         return compareFilterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5109,22 +5109,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FamilyFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FamilyFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5136,7 +5136,7 @@ public final class FilterProtos {
 
   public interface FilterListOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FilterList)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.FilterList.Operator operator = 1;
@@ -5175,11 +5175,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.FilterList}
    */
   public  static final class FilterList extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FilterList)
       FilterListOrBuilder {
     // Use FilterList.newBuilder() to construct.
-    private FilterList(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FilterList(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FilterList() {
@@ -5188,18 +5188,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FilterList(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5237,10 +5237,10 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -5250,12 +5250,12 @@ public final class FilterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterList_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterList_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5266,7 +5266,7 @@ public final class FilterProtos {
      * Protobuf enum {@code hbase.pb.FilterList.Operator}
      */
     public enum Operator
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * MUST_PASS_ALL = 1;
        */
@@ -5307,27 +5307,27 @@ public final class FilterProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Operator> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Operator findValueByNumber(int number) {
                 return Operator.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList.getDescriptor().getEnumTypes().get(0);
       }
@@ -5335,7 +5335,7 @@ public final class FilterProtos {
       private static final Operator[] VALUES = values();
 
       public static Operator valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -5424,7 +5424,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, operator_);
@@ -5441,11 +5441,11 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, operator_);
       }
       for (int i = 0; i < filters_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, filters_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -5496,61 +5496,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5568,7 +5568,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5576,15 +5576,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FilterList}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FilterList)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterListOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterList_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterList_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5597,12 +5597,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getFiltersFieldBuilder();
         }
@@ -5620,7 +5620,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterList_descriptor;
       }
@@ -5663,29 +5663,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList)other);
         } else {
@@ -5718,7 +5718,7 @@ public final class FilterProtos {
               filters_ = other.filters_;
               bitField0_ = (bitField0_ & ~0x00000002);
               filtersBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getFiltersFieldBuilder() : null;
             } else {
               filtersBuilder_.addAllMessages(other.filters_);
@@ -5743,13 +5743,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterList) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5806,7 +5806,7 @@ public final class FilterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> filtersBuilder_;
 
       /**
@@ -5938,7 +5938,7 @@ public final class FilterProtos {
           java.lang.Iterable values) {
         if (filtersBuilder_ == null) {
           ensureFiltersIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, filters_);
           onChanged();
         } else {
@@ -6022,11 +6022,11 @@ public final class FilterProtos {
            getFiltersBuilderList() {
         return getFiltersFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> 
           getFiltersFieldBuilder() {
         if (filtersBuilder_ == null) {
-          filtersBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          filtersBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder>(
                   filters_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -6037,12 +6037,12 @@ public final class FilterProtos {
         return filtersBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6060,22 +6060,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FilterList parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FilterList(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6087,7 +6087,7 @@ public final class FilterProtos {
 
   public interface FilterWrapperOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FilterWrapper)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.Filter filter = 1;
@@ -6106,29 +6106,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.FilterWrapper}
    */
   public  static final class FilterWrapper extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FilterWrapper)
       FilterWrapperOrBuilder {
     // Use FilterWrapper.newBuilder() to construct.
-    private FilterWrapper(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FilterWrapper(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FilterWrapper() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FilterWrapper(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6159,22 +6159,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterWrapper_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterWrapper_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6221,7 +6221,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getFilter());
@@ -6235,7 +6235,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -6281,61 +6281,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6353,7 +6353,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6361,15 +6361,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FilterWrapper}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FilterWrapper)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapperOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterWrapper_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterWrapper_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6382,12 +6382,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getFilterFieldBuilder();
         }
@@ -6403,7 +6403,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterWrapper_descriptor;
       }
@@ -6441,29 +6441,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper)other);
         } else {
@@ -6493,13 +6493,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterWrapper) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6512,7 +6512,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter filter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> filterBuilder_;
       /**
        * required .hbase.pb.Filter filter = 1;
@@ -6615,11 +6615,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.Filter filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> 
           getFilterFieldBuilder() {
         if (filterBuilder_ == null) {
-          filterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          filterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder>(
                   getFilter(),
                   getParentForChildren(),
@@ -6629,12 +6629,12 @@ public final class FilterProtos {
         return filterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6652,22 +6652,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FilterWrapper parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FilterWrapper(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6679,34 +6679,34 @@ public final class FilterProtos {
 
   public interface FirstKeyOnlyFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FirstKeyOnlyFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.FirstKeyOnlyFilter}
    */
   public  static final class FirstKeyOnlyFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FirstKeyOnlyFilter)
       FirstKeyOnlyFilterOrBuilder {
     // Use FirstKeyOnlyFilter.newBuilder() to construct.
-    private FirstKeyOnlyFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FirstKeyOnlyFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FirstKeyOnlyFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FirstKeyOnlyFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6724,22 +6724,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyOnlyFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyOnlyFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6756,7 +6756,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -6800,61 +6800,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6872,7 +6872,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6880,15 +6880,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FirstKeyOnlyFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FirstKeyOnlyFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyOnlyFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyOnlyFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6901,12 +6901,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -6915,7 +6915,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyOnlyFilter_descriptor;
       }
@@ -6942,29 +6942,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter)other);
         } else {
@@ -6985,13 +6985,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyOnlyFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7002,12 +7002,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7025,22 +7025,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FirstKeyOnlyFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FirstKeyOnlyFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7052,12 +7052,12 @@ public final class FilterProtos {
 
   public interface FirstKeyValueMatchingQualifiersFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FirstKeyValueMatchingQualifiersFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated bytes qualifiers = 1;
      */
-    java.util.List getQualifiersList();
+    java.util.List getQualifiersList();
     /**
      * repeated bytes qualifiers = 1;
      */
@@ -7065,17 +7065,17 @@ public final class FilterProtos {
     /**
      * repeated bytes qualifiers = 1;
      */
-    com.google.protobuf.ByteString getQualifiers(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifiers(int index);
   }
   /**
    * Protobuf type {@code hbase.pb.FirstKeyValueMatchingQualifiersFilter}
    */
   public  static final class FirstKeyValueMatchingQualifiersFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FirstKeyValueMatchingQualifiersFilter)
       FirstKeyValueMatchingQualifiersFilterOrBuilder {
     // Use FirstKeyValueMatchingQualifiersFilter.newBuilder() to construct.
-    private FirstKeyValueMatchingQualifiersFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FirstKeyValueMatchingQualifiersFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FirstKeyValueMatchingQualifiersFilter() {
@@ -7083,18 +7083,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FirstKeyValueMatchingQualifiersFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7112,7 +7112,7 @@ public final class FilterProtos {
             }
             case 10: {
               if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                qualifiers_ = new java.util.ArrayList();
+                qualifiers_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000001;
               }
               qualifiers_.add(input.readBytes());
@@ -7120,10 +7120,10 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -7133,12 +7133,12 @@ public final class FilterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7146,11 +7146,11 @@ public final class FilterProtos {
     }
 
     public static final int QUALIFIERS_FIELD_NUMBER = 1;
-    private java.util.List qualifiers_;
+    private java.util.List qualifiers_;
     /**
      * repeated bytes qualifiers = 1;
      */
-    public java.util.List
+    public java.util.List
         getQualifiersList() {
       return qualifiers_;
     }
@@ -7163,7 +7163,7 @@ public final class FilterProtos {
     /**
      * repeated bytes qualifiers = 1;
      */
-    public com.google.protobuf.ByteString getQualifiers(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifiers(int index) {
       return qualifiers_.get(index);
     }
 
@@ -7177,7 +7177,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < qualifiers_.size(); i++) {
         output.writeBytes(1, qualifiers_.get(i));
@@ -7193,7 +7193,7 @@ public final class FilterProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < qualifiers_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(qualifiers_.get(i));
         }
         size += dataSize;
@@ -7239,61 +7239,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7311,7 +7311,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7319,15 +7319,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FirstKeyValueMatchingQualifiersFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FirstKeyValueMatchingQualifiersFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7340,12 +7340,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7356,7 +7356,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_descriptor;
       }
@@ -7389,29 +7389,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter)other);
         } else {
@@ -7442,13 +7442,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7460,17 +7460,17 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private java.util.List qualifiers_ = java.util.Collections.emptyList();
+      private java.util.List qualifiers_ = java.util.Collections.emptyList();
       private void ensureQualifiersIsMutable() {
         if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          qualifiers_ = new java.util.ArrayList(qualifiers_);
+          qualifiers_ = new java.util.ArrayList(qualifiers_);
           bitField0_ |= 0x00000001;
          }
       }
       /**
        * repeated bytes qualifiers = 1;
        */
-      public java.util.List
+      public java.util.List
           getQualifiersList() {
         return java.util.Collections.unmodifiableList(qualifiers_);
       }
@@ -7483,14 +7483,14 @@ public final class FilterProtos {
       /**
        * repeated bytes qualifiers = 1;
        */
-      public com.google.protobuf.ByteString getQualifiers(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifiers(int index) {
         return qualifiers_.get(index);
       }
       /**
        * repeated bytes qualifiers = 1;
        */
       public Builder setQualifiers(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -7502,7 +7502,7 @@ public final class FilterProtos {
       /**
        * repeated bytes qualifiers = 1;
        */
-      public Builder addQualifiers(com.google.protobuf.ByteString value) {
+      public Builder addQualifiers(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -7515,9 +7515,9 @@ public final class FilterProtos {
        * repeated bytes qualifiers = 1;
        */
       public Builder addAllQualifiers(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureQualifiersIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, qualifiers_);
         onChanged();
         return this;
@@ -7532,12 +7532,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7555,22 +7555,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FirstKeyValueMatchingQualifiersFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FirstKeyValueMatchingQualifiersFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7582,7 +7582,7 @@ public final class FilterProtos {
 
   public interface FuzzyRowFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FuzzyRowFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.BytesBytesPair fuzzy_keys_data = 1;
@@ -7612,11 +7612,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.FuzzyRowFilter}
    */
   public  static final class FuzzyRowFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FuzzyRowFilter)
       FuzzyRowFilterOrBuilder {
     // Use FuzzyRowFilter.newBuilder() to construct.
-    private FuzzyRowFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FuzzyRowFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FuzzyRowFilter() {
@@ -7624,18 +7624,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FuzzyRowFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7662,10 +7662,10 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -7675,12 +7675,12 @@ public final class FilterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FuzzyRowFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FuzzyRowFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7738,7 +7738,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < fuzzyKeysData_.size(); i++) {
         output.writeMessage(1, fuzzyKeysData_.get(i));
@@ -7752,7 +7752,7 @@ public final class FilterProtos {
 
       size = 0;
       for (int i = 0; i < fuzzyKeysData_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, fuzzyKeysData_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -7795,61 +7795,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7867,7 +7867,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7875,15 +7875,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FuzzyRowFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FuzzyRowFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FuzzyRowFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FuzzyRowFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7896,12 +7896,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getFuzzyKeysDataFieldBuilder();
         }
@@ -7917,7 +7917,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FuzzyRowFilter_descriptor;
       }
@@ -7954,29 +7954,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter)other);
         } else {
@@ -8006,7 +8006,7 @@ public final class FilterProtos {
               fuzzyKeysData_ = other.fuzzyKeysData_;
               bitField0_ = (bitField0_ & ~0x00000001);
               fuzzyKeysDataBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getFuzzyKeysDataFieldBuilder() : null;
             } else {
               fuzzyKeysDataBuilder_.addAllMessages(other.fuzzyKeysData_);
@@ -8028,13 +8028,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FuzzyRowFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8055,7 +8055,7 @@ public final class FilterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> fuzzyKeysDataBuilder_;
 
       /**
@@ -8187,7 +8187,7 @@ public final class FilterProtos {
           java.lang.Iterable values) {
         if (fuzzyKeysDataBuilder_ == null) {
           ensureFuzzyKeysDataIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, fuzzyKeysData_);
           onChanged();
         } else {
@@ -8271,11 +8271,11 @@ public final class FilterProtos {
            getFuzzyKeysDataBuilderList() {
         return getFuzzyKeysDataFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
           getFuzzyKeysDataFieldBuilder() {
         if (fuzzyKeysDataBuilder_ == null) {
-          fuzzyKeysDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          fuzzyKeysDataBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>(
                   fuzzyKeysData_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -8286,12 +8286,12 @@ public final class FilterProtos {
         return fuzzyKeysDataBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8309,22 +8309,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FuzzyRowFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FuzzyRowFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8336,7 +8336,7 @@ public final class FilterProtos {
 
   public interface InclusiveStopFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.InclusiveStopFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes stop_row_key = 1;
@@ -8345,36 +8345,36 @@ public final class FilterProtos {
     /**
      * optional bytes stop_row_key = 1;
      */
-    com.google.protobuf.ByteString getStopRowKey();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRowKey();
   }
   /**
    * Protobuf type {@code hbase.pb.InclusiveStopFilter}
    */
   public  static final class InclusiveStopFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.InclusiveStopFilter)
       InclusiveStopFilterOrBuilder {
     // Use InclusiveStopFilter.newBuilder() to construct.
-    private InclusiveStopFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private InclusiveStopFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private InclusiveStopFilter() {
-      stopRowKey_ = com.google.protobuf.ByteString.EMPTY;
+      stopRowKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private InclusiveStopFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8397,22 +8397,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_InclusiveStopFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_InclusiveStopFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8421,7 +8421,7 @@ public final class FilterProtos {
 
     private int bitField0_;
     public static final int STOP_ROW_KEY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString stopRowKey_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stopRowKey_;
     /**
      * optional bytes stop_row_key = 1;
      */
@@ -8431,7 +8431,7 @@ public final class FilterProtos {
     /**
      * optional bytes stop_row_key = 1;
      */
-    public com.google.protobuf.ByteString getStopRowKey() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRowKey() {
       return stopRowKey_;
     }
 
@@ -8445,7 +8445,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, stopRowKey_);
@@ -8459,7 +8459,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, stopRowKey_);
       }
       size += unknownFields.getSerializedSize();
@@ -8505,61 +8505,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8577,7 +8577,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8585,15 +8585,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.InclusiveStopFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.InclusiveStopFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_InclusiveStopFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_InclusiveStopFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8606,23 +8606,23 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        stopRowKey_ = com.google.protobuf.ByteString.EMPTY;
+        stopRowKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_InclusiveStopFilter_descriptor;
       }
@@ -8656,29 +8656,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter)other);
         } else {
@@ -8702,13 +8702,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.InclusiveStopFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8720,7 +8720,7 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString stopRowKey_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stopRowKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes stop_row_key = 1;
        */
@@ -8730,13 +8730,13 @@ public final class FilterProtos {
       /**
        * optional bytes stop_row_key = 1;
        */
-      public com.google.protobuf.ByteString getStopRowKey() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRowKey() {
         return stopRowKey_;
       }
       /**
        * optional bytes stop_row_key = 1;
        */
-      public Builder setStopRowKey(com.google.protobuf.ByteString value) {
+      public Builder setStopRowKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8755,12 +8755,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8778,22 +8778,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public InclusiveStopFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new InclusiveStopFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8805,7 +8805,7 @@ public final class FilterProtos {
 
   public interface KeyOnlyFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.KeyOnlyFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool len_as_val = 1;
@@ -8820,11 +8820,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.KeyOnlyFilter}
    */
   public  static final class KeyOnlyFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.KeyOnlyFilter)
       KeyOnlyFilterOrBuilder {
     // Use KeyOnlyFilter.newBuilder() to construct.
-    private KeyOnlyFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private KeyOnlyFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private KeyOnlyFilter() {
@@ -8832,18 +8832,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private KeyOnlyFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8866,22 +8866,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_KeyOnlyFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_KeyOnlyFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8918,7 +8918,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, lenAsVal_);
@@ -8932,7 +8932,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, lenAsVal_);
       }
       size += unknownFields.getSerializedSize();
@@ -8970,7 +8970,7 @@ public final class FilterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLenAsVal()) {
         hash = (37 * hash) + LEN_AS_VAL_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getLenAsVal());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -8979,61 +8979,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9051,7 +9051,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9059,15 +9059,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.KeyOnlyFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.KeyOnlyFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_KeyOnlyFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_KeyOnlyFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9080,12 +9080,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -9096,7 +9096,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_KeyOnlyFilter_descriptor;
       }
@@ -9130,29 +9130,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter)other);
         } else {
@@ -9179,13 +9179,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.KeyOnlyFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9229,12 +9229,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9252,22 +9252,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public KeyOnlyFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new KeyOnlyFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9279,12 +9279,12 @@ public final class FilterProtos {
 
   public interface MultipleColumnPrefixFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MultipleColumnPrefixFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated bytes sorted_prefixes = 1;
      */
-    java.util.List getSortedPrefixesList();
+    java.util.List getSortedPrefixesList();
     /**
      * repeated bytes sorted_prefixes = 1;
      */
@@ -9292,17 +9292,17 @@ public final class FilterProtos {
     /**
      * repeated bytes sorted_prefixes = 1;
      */
-    com.google.protobuf.ByteString getSortedPrefixes(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSortedPrefixes(int index);
   }
   /**
    * Protobuf type {@code hbase.pb.MultipleColumnPrefixFilter}
    */
   public  static final class MultipleColumnPrefixFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MultipleColumnPrefixFilter)
       MultipleColumnPrefixFilterOrBuilder {
     // Use MultipleColumnPrefixFilter.newBuilder() to construct.
-    private MultipleColumnPrefixFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MultipleColumnPrefixFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MultipleColumnPrefixFilter() {
@@ -9310,18 +9310,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MultipleColumnPrefixFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9339,7 +9339,7 @@ public final class FilterProtos {
             }
             case 10: {
               if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                sortedPrefixes_ = new java.util.ArrayList();
+                sortedPrefixes_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000001;
               }
               sortedPrefixes_.add(input.readBytes());
@@ -9347,10 +9347,10 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -9360,12 +9360,12 @@ public final class FilterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultipleColumnPrefixFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultipleColumnPrefixFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9373,11 +9373,11 @@ public final class FilterProtos {
     }
 
     public static final int SORTED_PREFIXES_FIELD_NUMBER = 1;
-    private java.util.List sortedPrefixes_;
+    private java.util.List sortedPrefixes_;
     /**
      * repeated bytes sorted_prefixes = 1;
      */
-    public java.util.List
+    public java.util.List
         getSortedPrefixesList() {
       return sortedPrefixes_;
     }
@@ -9390,7 +9390,7 @@ public final class FilterProtos {
     /**
      * repeated bytes sorted_prefixes = 1;
      */
-    public com.google.protobuf.ByteString getSortedPrefixes(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSortedPrefixes(int index) {
       return sortedPrefixes_.get(index);
     }
 
@@ -9404,7 +9404,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < sortedPrefixes_.size(); i++) {
         output.writeBytes(1, sortedPrefixes_.get(i));
@@ -9420,7 +9420,7 @@ public final class FilterProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < sortedPrefixes_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(sortedPrefixes_.get(i));
         }
         size += dataSize;
@@ -9466,61 +9466,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9538,7 +9538,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9546,15 +9546,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.MultipleColumnPrefixFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MultipleColumnPrefixFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultipleColumnPrefixFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultipleColumnPrefixFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9567,12 +9567,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -9583,7 +9583,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultipleColumnPrefixFilter_descriptor;
       }
@@ -9616,29 +9616,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter)other);
         } else {
@@ -9669,13 +9669,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9687,17 +9687,17 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private java.util.List sortedPrefixes_ = java.util.Collections.emptyList();
+      private java.util.List sortedPrefixes_ = java.util.Collections.emptyList();
       private void ensureSortedPrefixesIsMutable() {
         if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          sortedPrefixes_ = new java.util.ArrayList(sortedPrefixes_);
+          sortedPrefixes_ = new java.util.ArrayList(sortedPrefixes_);
           bitField0_ |= 0x00000001;
          }
       }
       /**
        * repeated bytes sorted_prefixes = 1;
        */
-      public java.util.List
+      public java.util.List
           getSortedPrefixesList() {
         return java.util.Collections.unmodifiableList(sortedPrefixes_);
       }
@@ -9710,14 +9710,14 @@ public final class FilterProtos {
       /**
        * repeated bytes sorted_prefixes = 1;
        */
-      public com.google.protobuf.ByteString getSortedPrefixes(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSortedPrefixes(int index) {
         return sortedPrefixes_.get(index);
       }
       /**
        * repeated bytes sorted_prefixes = 1;
        */
       public Builder setSortedPrefixes(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -9729,7 +9729,7 @@ public final class FilterProtos {
       /**
        * repeated bytes sorted_prefixes = 1;
        */
-      public Builder addSortedPrefixes(com.google.protobuf.ByteString value) {
+      public Builder addSortedPrefixes(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -9742,9 +9742,9 @@ public final class FilterProtos {
        * repeated bytes sorted_prefixes = 1;
        */
       public Builder addAllSortedPrefixes(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureSortedPrefixesIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, sortedPrefixes_);
         onChanged();
         return this;
@@ -9759,12 +9759,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9782,22 +9782,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MultipleColumnPrefixFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MultipleColumnPrefixFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9809,7 +9809,7 @@ public final class FilterProtos {
 
   public interface PageFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.PageFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int64 page_size = 1;
@@ -9824,11 +9824,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.PageFilter}
    */
   public  static final class PageFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.PageFilter)
       PageFilterOrBuilder {
     // Use PageFilter.newBuilder() to construct.
-    private PageFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private PageFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private PageFilter() {
@@ -9836,18 +9836,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private PageFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9870,22 +9870,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PageFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PageFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9922,7 +9922,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, pageSize_);
@@ -9936,7 +9936,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, pageSize_);
       }
       size += unknownFields.getSerializedSize();
@@ -9974,7 +9974,7 @@ public final class FilterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasPageSize()) {
         hash = (37 * hash) + PAGE_SIZE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getPageSize());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -9983,61 +9983,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10055,7 +10055,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10063,15 +10063,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.PageFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.PageFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PageFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PageFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10084,12 +10084,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -10100,7 +10100,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PageFilter_descriptor;
       }
@@ -10134,29 +10134,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter)other);
         } else {
@@ -10183,13 +10183,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PageFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -10233,12 +10233,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10256,22 +10256,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public PageFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new PageFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10283,7 +10283,7 @@ public final class FilterProtos {
 
   public interface PrefixFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.PrefixFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes prefix = 1;
@@ -10292,36 +10292,36 @@ public final class FilterProtos {
     /**
      * optional bytes prefix = 1;
      */
-    com.google.protobuf.ByteString getPrefix();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPrefix();
   }
   /**
    * Protobuf type {@code hbase.pb.PrefixFilter}
    */
   public  static final class PrefixFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.PrefixFilter)
       PrefixFilterOrBuilder {
     // Use PrefixFilter.newBuilder() to construct.
-    private PrefixFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private PrefixFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private PrefixFilter() {
-      prefix_ = com.google.protobuf.ByteString.EMPTY;
+      prefix_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private PrefixFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10344,22 +10344,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PrefixFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PrefixFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10368,7 +10368,7 @@ public final class FilterProtos {
 
     private int bitField0_;
     public static final int PREFIX_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString prefix_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString prefix_;
     /**
      * optional bytes prefix = 1;
      */
@@ -10378,7 +10378,7 @@ public final class FilterProtos {
     /**
      * optional bytes prefix = 1;
      */
-    public com.google.protobuf.ByteString getPrefix() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPrefix() {
       return prefix_;
     }
 
@@ -10392,7 +10392,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, prefix_);
@@ -10406,7 +10406,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, prefix_);
       }
       size += unknownFields.getSerializedSize();
@@ -10452,61 +10452,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10524,7 +10524,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10532,15 +10532,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.PrefixFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.PrefixFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PrefixFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PrefixFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10553,23 +10553,23 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        prefix_ = com.google.protobuf.ByteString.EMPTY;
+        prefix_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_PrefixFilter_descriptor;
       }
@@ -10603,29 +10603,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter)other);
         } else {
@@ -10649,13 +10649,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.PrefixFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -10667,7 +10667,7 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString prefix_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString prefix_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes prefix = 1;
        */
@@ -10677,13 +10677,13 @@ public final class FilterProtos {
       /**
        * optional bytes prefix = 1;
        */
-      public com.google.protobuf.ByteString getPrefix() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getPrefix() {
         return prefix_;
       }
       /**
        * optional bytes prefix = 1;
        */
-      public Builder setPrefix(com.google.protobuf.ByteString value) {
+      public Builder setPrefix(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10702,12 +10702,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10725,22 +10725,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public PrefixFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new PrefixFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10752,7 +10752,7 @@ public final class FilterProtos {
 
   public interface QualifierFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.QualifierFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -10771,29 +10771,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.QualifierFilter}
    */
   public  static final class QualifierFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.QualifierFilter)
       QualifierFilterOrBuilder {
     // Use QualifierFilter.newBuilder() to construct.
-    private QualifierFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private QualifierFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private QualifierFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private QualifierFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10824,22 +10824,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_QualifierFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_QualifierFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10886,7 +10886,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getCompareFilter());
@@ -10900,7 +10900,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getCompareFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -10946,61 +10946,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11018,7 +11018,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11026,15 +11026,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.QualifierFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.QualifierFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_QualifierFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_QualifierFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11047,12 +11047,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getCompareFilterFieldBuilder();
         }
@@ -11068,7 +11068,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_QualifierFilter_descriptor;
       }
@@ -11106,29 +11106,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter)other);
         } else {
@@ -11158,13 +11158,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.QualifierFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11177,7 +11177,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter compareFilter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> compareFilterBuilder_;
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -11280,11 +11280,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> 
           getCompareFilterFieldBuilder() {
         if (compareFilterBuilder_ == null) {
-          compareFilterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          compareFilterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder>(
                   getCompareFilter(),
                   getParentForChildren(),
@@ -11294,12 +11294,12 @@ public final class FilterProtos {
         return compareFilterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11317,22 +11317,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public QualifierFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new QualifierFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11344,7 +11344,7 @@ public final class FilterProtos {
 
   public interface RandomRowFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RandomRowFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required float chance = 1;
@@ -11359,11 +11359,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.RandomRowFilter}
    */
   public  static final class RandomRowFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RandomRowFilter)
       RandomRowFilterOrBuilder {
     // Use RandomRowFilter.newBuilder() to construct.
-    private RandomRowFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RandomRowFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RandomRowFilter() {
@@ -11371,18 +11371,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RandomRowFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11405,22 +11405,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RandomRowFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RandomRowFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11457,7 +11457,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeFloat(1, chance_);
@@ -11471,7 +11471,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeFloatSize(1, chance_);
       }
       size += unknownFields.getSerializedSize();
@@ -11520,61 +11520,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11592,7 +11592,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11600,15 +11600,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.RandomRowFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RandomRowFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RandomRowFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RandomRowFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11621,12 +11621,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -11637,7 +11637,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RandomRowFilter_descriptor;
       }
@@ -11671,29 +11671,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter)other);
         } else {
@@ -11720,13 +11720,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RandomRowFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11770,12 +11770,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11793,22 +11793,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RandomRowFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RandomRowFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11820,7 +11820,7 @@ public final class FilterProtos {
 
   public interface RowFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RowFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -11839,29 +11839,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.RowFilter}
    */
   public  static final class RowFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RowFilter)
       RowFilterOrBuilder {
     // Use RowFilter.newBuilder() to construct.
-    private RowFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RowFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RowFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RowFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11892,22 +11892,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11954,7 +11954,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getCompareFilter());
@@ -11968,7 +11968,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getCompareFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -12014,61 +12014,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12086,7 +12086,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12094,15 +12094,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.RowFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RowFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12115,12 +12115,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getCompareFilterFieldBuilder();
         }
@@ -12136,7 +12136,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowFilter_descriptor;
       }
@@ -12174,29 +12174,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter)other);
         } else {
@@ -12226,13 +12226,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12245,7 +12245,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter compareFilter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> compareFilterBuilder_;
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -12348,11 +12348,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> 
           getCompareFilterFieldBuilder() {
         if (compareFilterBuilder_ == null) {
-          compareFilterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          compareFilterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder>(
                   getCompareFilter(),
                   getParentForChildren(),
@@ -12362,12 +12362,12 @@ public final class FilterProtos {
         return compareFilterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12385,22 +12385,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RowFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RowFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12412,7 +12412,7 @@ public final class FilterProtos {
 
   public interface SingleColumnValueExcludeFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SingleColumnValueExcludeFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.SingleColumnValueFilter single_column_value_filter = 1;
@@ -12431,29 +12431,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.SingleColumnValueExcludeFilter}
    */
   public  static final class SingleColumnValueExcludeFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SingleColumnValueExcludeFilter)
       SingleColumnValueExcludeFilterOrBuilder {
     // Use SingleColumnValueExcludeFilter.newBuilder() to construct.
-    private SingleColumnValueExcludeFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SingleColumnValueExcludeFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SingleColumnValueExcludeFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SingleColumnValueExcludeFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12484,22 +12484,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueExcludeFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueExcludeFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12546,7 +12546,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSingleColumnValueFilter());
@@ -12560,7 +12560,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSingleColumnValueFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -12606,61 +12606,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12678,7 +12678,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12686,15 +12686,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.SingleColumnValueExcludeFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SingleColumnValueExcludeFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueExcludeFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueExcludeFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12707,12 +12707,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSingleColumnValueFilterFieldBuilder();
         }
@@ -12728,7 +12728,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueExcludeFilter_descriptor;
       }
@@ -12766,29 +12766,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter)other);
         } else {
@@ -12818,13 +12818,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12837,7 +12837,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter singleColumnValueFilter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilterOrBuilder> singleColumnValueFilterBuilder_;
       /**
        * required .hbase.pb.SingleColumnValueFilter single_column_value_filter = 1;
@@ -12940,11 +12940,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.SingleColumnValueFilter single_column_value_filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilterOrBuilder> 
           getSingleColumnValueFilterFieldBuilder() {
         if (singleColumnValueFilterBuilder_ == null) {
-          singleColumnValueFilterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          singleColumnValueFilterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilterOrBuilder>(
                   getSingleColumnValueFilter(),
                   getParentForChildren(),
@@ -12954,12 +12954,12 @@ public final class FilterProtos {
         return singleColumnValueFilterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12977,22 +12977,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SingleColumnValueExcludeFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SingleColumnValueExcludeFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13004,7 +13004,7 @@ public final class FilterProtos {
 
   public interface SingleColumnValueFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SingleColumnValueFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes column_family = 1;
@@ -13013,7 +13013,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_family = 1;
      */
-    com.google.protobuf.ByteString getColumnFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily();
 
     /**
      * optional bytes column_qualifier = 2;
@@ -13022,7 +13022,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_qualifier = 2;
      */
-    com.google.protobuf.ByteString getColumnQualifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnQualifier();
 
     /**
      * required .hbase.pb.CompareType compare_op = 3;
@@ -13068,34 +13068,34 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.SingleColumnValueFilter}
    */
   public  static final class SingleColumnValueFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SingleColumnValueFilter)
       SingleColumnValueFilterOrBuilder {
     // Use SingleColumnValueFilter.newBuilder() to construct.
-    private SingleColumnValueFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SingleColumnValueFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SingleColumnValueFilter() {
-      columnFamily_ = com.google.protobuf.ByteString.EMPTY;
-      columnQualifier_ = com.google.protobuf.ByteString.EMPTY;
+      columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      columnQualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       compareOp_ = 0;
       filterIfMissing_ = false;
       latestVersionOnly_ = false;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SingleColumnValueFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -13157,22 +13157,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -13181,7 +13181,7 @@ public final class FilterProtos {
 
     private int bitField0_;
     public static final int COLUMN_FAMILY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString columnFamily_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnFamily_;
     /**
      * optional bytes column_family = 1;
      */
@@ -13191,12 +13191,12 @@ public final class FilterProtos {
     /**
      * optional bytes column_family = 1;
      */
-    public com.google.protobuf.ByteString getColumnFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily() {
       return columnFamily_;
     }
 
     public static final int COLUMN_QUALIFIER_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString columnQualifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnQualifier_;
     /**
      * optional bytes column_qualifier = 2;
      */
@@ -13206,7 +13206,7 @@ public final class FilterProtos {
     /**
      * optional bytes column_qualifier = 2;
      */
-    public com.google.protobuf.ByteString getColumnQualifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnQualifier() {
       return columnQualifier_;
     }
 
@@ -13299,7 +13299,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, columnFamily_);
@@ -13328,27 +13328,27 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, columnFamily_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, columnQualifier_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(3, compareOp_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getComparator());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, filterIfMissing_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(6, latestVersionOnly_);
       }
       size += unknownFields.getSerializedSize();
@@ -13426,12 +13426,12 @@ public final class FilterProtos {
       }
       if (hasFilterIfMissing()) {
         hash = (37 * hash) + FILTER_IF_MISSING_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getFilterIfMissing());
       }
       if (hasLatestVersionOnly()) {
         hash = (37 * hash) + LATEST_VERSION_ONLY_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getLatestVersionOnly());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -13440,61 +13440,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -13512,7 +13512,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -13520,15 +13520,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.SingleColumnValueFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SingleColumnValueFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13541,21 +13541,21 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getComparatorFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+        columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        columnQualifier_ = com.google.protobuf.ByteString.EMPTY;
+        columnQualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         compareOp_ = 0;
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -13572,7 +13572,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SingleColumnValueFilter_descriptor;
       }
@@ -13630,29 +13630,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter)other);
         } else {
@@ -13700,13 +13700,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SingleColumnValueFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13718,7 +13718,7 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes column_family = 1;
        */
@@ -13728,13 +13728,13 @@ public final class FilterProtos {
       /**
        * optional bytes column_family = 1;
        */
-      public com.google.protobuf.ByteString getColumnFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily() {
         return columnFamily_;
       }
       /**
        * optional bytes column_family = 1;
        */
-      public Builder setColumnFamily(com.google.protobuf.ByteString value) {
+      public Builder setColumnFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -13753,7 +13753,7 @@ public final class FilterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString columnQualifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnQualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes column_qualifier = 2;
        */
@@ -13763,13 +13763,13 @@ public final class FilterProtos {
       /**
        * optional bytes column_qualifier = 2;
        */
-      public com.google.protobuf.ByteString getColumnQualifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnQualifier() {
         return columnQualifier_;
       }
       /**
        * optional bytes column_qualifier = 2;
        */
-      public Builder setColumnQualifier(com.google.protobuf.ByteString value) {
+      public Builder setColumnQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -13825,7 +13825,7 @@ public final class FilterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator comparator_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> comparatorBuilder_;
       /**
        * required .hbase.pb.Comparator comparator = 4;
@@ -13928,11 +13928,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.Comparator comparator = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> 
           getComparatorFieldBuilder() {
         if (comparatorBuilder_ == null) {
-          comparatorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          comparatorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.ComparatorOrBuilder>(
                   getComparator(),
                   getParentForChildren(),
@@ -14006,12 +14006,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14029,22 +14029,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SingleColumnValueFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SingleColumnValueFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14056,7 +14056,7 @@ public final class FilterProtos {
 
   public interface SkipFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SkipFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.Filter filter = 1;
@@ -14075,29 +14075,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.SkipFilter}
    */
   public  static final class SkipFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SkipFilter)
       SkipFilterOrBuilder {
     // Use SkipFilter.newBuilder() to construct.
-    private SkipFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SkipFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SkipFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SkipFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14128,22 +14128,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SkipFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SkipFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14190,7 +14190,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getFilter());
@@ -14204,7 +14204,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -14250,61 +14250,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14322,7 +14322,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -14330,15 +14330,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.SkipFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SkipFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SkipFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SkipFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -14351,12 +14351,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getFilterFieldBuilder();
         }
@@ -14372,7 +14372,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_SkipFilter_descriptor;
       }
@@ -14410,29 +14410,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter)other);
         } else {
@@ -14462,13 +14462,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.SkipFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -14481,7 +14481,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter filter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> filterBuilder_;
       /**
        * required .hbase.pb.Filter filter = 1;
@@ -14584,11 +14584,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.Filter filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> 
           getFilterFieldBuilder() {
         if (filterBuilder_ == null) {
-          filterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          filterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder>(
                   getFilter(),
                   getParentForChildren(),
@@ -14598,12 +14598,12 @@ public final class FilterProtos {
         return filterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14621,22 +14621,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SkipFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SkipFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14648,7 +14648,7 @@ public final class FilterProtos {
 
   public interface TimestampsFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TimestampsFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated int64 timestamps = 1 [packed = true];
@@ -14676,11 +14676,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.TimestampsFilter}
    */
   public  static final class TimestampsFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TimestampsFilter)
       TimestampsFilterOrBuilder {
     // Use TimestampsFilter.newBuilder() to construct.
-    private TimestampsFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TimestampsFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TimestampsFilter() {
@@ -14689,18 +14689,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TimestampsFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14744,10 +14744,10 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -14757,12 +14757,12 @@ public final class FilterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_TimestampsFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_TimestampsFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14818,7 +14818,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
       if (getTimestampsList().size() > 0) {
@@ -14842,19 +14842,19 @@ public final class FilterProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < timestamps_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeInt64SizeNoTag(timestamps_.get(i));
         }
         size += dataSize;
         if (!getTimestampsList().isEmpty()) {
           size += 1;
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeInt32SizeNoTag(dataSize);
         }
         timestampsMemoizedSerializedSize = dataSize;
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, canHint_);
       }
       size += unknownFields.getSerializedSize();
@@ -14898,7 +14898,7 @@ public final class FilterProtos {
       }
       if (hasCanHint()) {
         hash = (37 * hash) + CAN_HINT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCanHint());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -14907,61 +14907,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14979,7 +14979,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -14987,15 +14987,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.TimestampsFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TimestampsFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_TimestampsFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_TimestampsFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15008,12 +15008,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -15026,7 +15026,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_TimestampsFilter_descriptor;
       }
@@ -15065,29 +15065,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter)other);
         } else {
@@ -15121,13 +15121,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.TimestampsFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15190,7 +15190,7 @@ public final class FilterProtos {
       public Builder addAllTimestamps(
           java.lang.Iterable values) {
         ensureTimestampsIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, timestamps_);
         onChanged();
         return this;
@@ -15237,12 +15237,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15260,22 +15260,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TimestampsFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TimestampsFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15287,7 +15287,7 @@ public final class FilterProtos {
 
   public interface ValueFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ValueFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -15306,29 +15306,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.ValueFilter}
    */
   public  static final class ValueFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ValueFilter)
       ValueFilterOrBuilder {
     // Use ValueFilter.newBuilder() to construct.
-    private ValueFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ValueFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ValueFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ValueFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15359,22 +15359,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ValueFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ValueFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -15421,7 +15421,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getCompareFilter());
@@ -15435,7 +15435,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getCompareFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -15481,61 +15481,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15553,7 +15553,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15561,15 +15561,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.ValueFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ValueFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ValueFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ValueFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15582,12 +15582,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getCompareFilterFieldBuilder();
         }
@@ -15603,7 +15603,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_ValueFilter_descriptor;
       }
@@ -15641,29 +15641,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter)other);
         } else {
@@ -15693,13 +15693,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.ValueFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15712,7 +15712,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter compareFilter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> compareFilterBuilder_;
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
@@ -15815,11 +15815,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.CompareFilter compare_filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder> 
           getCompareFilterFieldBuilder() {
         if (compareFilterBuilder_ == null) {
-          compareFilterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          compareFilterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.CompareFilterOrBuilder>(
                   getCompareFilter(),
                   getParentForChildren(),
@@ -15829,12 +15829,12 @@ public final class FilterProtos {
         return compareFilterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15852,22 +15852,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ValueFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ValueFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15879,7 +15879,7 @@ public final class FilterProtos {
 
   public interface WhileMatchFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WhileMatchFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.Filter filter = 1;
@@ -15898,29 +15898,29 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.WhileMatchFilter}
    */
   public  static final class WhileMatchFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WhileMatchFilter)
       WhileMatchFilterOrBuilder {
     // Use WhileMatchFilter.newBuilder() to construct.
-    private WhileMatchFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WhileMatchFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WhileMatchFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WhileMatchFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15951,22 +15951,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_WhileMatchFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_WhileMatchFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16013,7 +16013,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getFilter());
@@ -16027,7 +16027,7 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getFilter());
       }
       size += unknownFields.getSerializedSize();
@@ -16073,61 +16073,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16145,7 +16145,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16153,15 +16153,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.WhileMatchFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WhileMatchFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_WhileMatchFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_WhileMatchFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16174,12 +16174,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getFilterFieldBuilder();
         }
@@ -16195,7 +16195,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_WhileMatchFilter_descriptor;
       }
@@ -16233,29 +16233,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter)other);
         } else {
@@ -16285,13 +16285,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.WhileMatchFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16304,7 +16304,7 @@ public final class FilterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter filter_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> filterBuilder_;
       /**
        * required .hbase.pb.Filter filter = 1;
@@ -16407,11 +16407,11 @@ public final class FilterProtos {
       /**
        * required .hbase.pb.Filter filter = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder> 
           getFilterFieldBuilder() {
         if (filterBuilder_ == null) {
-          filterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          filterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.Filter.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterOrBuilder>(
                   getFilter(),
                   getParentForChildren(),
@@ -16421,12 +16421,12 @@ public final class FilterProtos {
         return filterBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16444,22 +16444,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WhileMatchFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WhileMatchFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16471,34 +16471,34 @@ public final class FilterProtos {
 
   public interface FilterAllFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FilterAllFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.FilterAllFilter}
    */
   public  static final class FilterAllFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FilterAllFilter)
       FilterAllFilterOrBuilder {
     // Use FilterAllFilter.newBuilder() to construct.
-    private FilterAllFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FilterAllFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FilterAllFilter() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FilterAllFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16516,22 +16516,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterAllFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterAllFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16548,7 +16548,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -16592,61 +16592,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16664,7 +16664,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16672,15 +16672,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.FilterAllFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FilterAllFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterAllFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterAllFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16693,12 +16693,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -16707,7 +16707,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_FilterAllFilter_descriptor;
       }
@@ -16734,29 +16734,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter)other);
         } else {
@@ -16777,13 +16777,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.FilterAllFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16794,12 +16794,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16817,22 +16817,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FilterAllFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FilterAllFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16844,7 +16844,7 @@ public final class FilterProtos {
 
   public interface RowRangeOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RowRange)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bytes start_row = 1;
@@ -16853,7 +16853,7 @@ public final class FilterProtos {
     /**
      * optional bytes start_row = 1;
      */
-    com.google.protobuf.ByteString getStartRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartRow();
 
     /**
      * optional bool start_row_inclusive = 2;
@@ -16871,7 +16871,7 @@ public final class FilterProtos {
     /**
      * optional bytes stop_row = 3;
      */
-    com.google.protobuf.ByteString getStopRow();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRow();
 
     /**
      * optional bool stop_row_inclusive = 4;
@@ -16886,33 +16886,33 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.RowRange}
    */
   public  static final class RowRange extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RowRange)
       RowRangeOrBuilder {
     // Use RowRange.newBuilder() to construct.
-    private RowRange(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RowRange(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RowRange() {
-      startRow_ = com.google.protobuf.ByteString.EMPTY;
+      startRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       startRowInclusive_ = false;
-      stopRow_ = com.google.protobuf.ByteString.EMPTY;
+      stopRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       stopRowInclusive_ = false;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RowRange(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16950,22 +16950,22 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowRange_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowRange_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16974,7 +16974,7 @@ public final class FilterProtos {
 
     private int bitField0_;
     public static final int START_ROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString startRow_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString startRow_;
     /**
      * optional bytes start_row = 1;
      */
@@ -16984,7 +16984,7 @@ public final class FilterProtos {
     /**
      * optional bytes start_row = 1;
      */
-    public com.google.protobuf.ByteString getStartRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartRow() {
       return startRow_;
     }
 
@@ -17004,7 +17004,7 @@ public final class FilterProtos {
     }
 
     public static final int STOP_ROW_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString stopRow_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stopRow_;
     /**
      * optional bytes stop_row = 3;
      */
@@ -17014,7 +17014,7 @@ public final class FilterProtos {
     /**
      * optional bytes stop_row = 3;
      */
-    public com.google.protobuf.ByteString getStopRow() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRow() {
       return stopRow_;
     }
 
@@ -17043,7 +17043,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, startRow_);
@@ -17066,19 +17066,19 @@ public final class FilterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, startRow_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, startRowInclusive_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, stopRow_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, stopRowInclusive_);
       }
       size += unknownFields.getSerializedSize();
@@ -17135,7 +17135,7 @@ public final class FilterProtos {
       }
       if (hasStartRowInclusive()) {
         hash = (37 * hash) + START_ROW_INCLUSIVE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getStartRowInclusive());
       }
       if (hasStopRow()) {
@@ -17144,7 +17144,7 @@ public final class FilterProtos {
       }
       if (hasStopRowInclusive()) {
         hash = (37 * hash) + STOP_ROW_INCLUSIVE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getStopRowInclusive());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -17153,61 +17153,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -17225,7 +17225,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -17233,15 +17233,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.RowRange}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RowRange)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRangeOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowRange_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowRange_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -17254,29 +17254,29 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        startRow_ = com.google.protobuf.ByteString.EMPTY;
+        startRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         startRowInclusive_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
-        stopRow_ = com.google.protobuf.ByteString.EMPTY;
+        stopRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         stopRowInclusive_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_RowRange_descriptor;
       }
@@ -17322,29 +17322,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange)other);
         } else {
@@ -17377,13 +17377,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -17395,7 +17395,7 @@ public final class FilterProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString startRow_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString startRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes start_row = 1;
        */
@@ -17405,13 +17405,13 @@ public final class FilterProtos {
       /**
        * optional bytes start_row = 1;
        */
-      public com.google.protobuf.ByteString getStartRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartRow() {
         return startRow_;
       }
       /**
        * optional bytes start_row = 1;
        */
-      public Builder setStartRow(com.google.protobuf.ByteString value) {
+      public Builder setStartRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -17462,7 +17462,7 @@ public final class FilterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString stopRow_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stopRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes stop_row = 3;
        */
@@ -17472,13 +17472,13 @@ public final class FilterProtos {
       /**
        * optional bytes stop_row = 3;
        */
-      public com.google.protobuf.ByteString getStopRow() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStopRow() {
         return stopRow_;
       }
       /**
        * optional bytes stop_row = 3;
        */
-      public Builder setStopRow(com.google.protobuf.ByteString value) {
+      public Builder setStopRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -17529,12 +17529,12 @@ public final class FilterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17552,22 +17552,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RowRange parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RowRange(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17579,7 +17579,7 @@ public final class FilterProtos {
 
   public interface MultiRowRangeFilterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MultiRowRangeFilter)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.RowRange row_range_list = 1;
@@ -17609,11 +17609,11 @@ public final class FilterProtos {
    * Protobuf type {@code hbase.pb.MultiRowRangeFilter}
    */
   public  static final class MultiRowRangeFilter extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MultiRowRangeFilter)
       MultiRowRangeFilterOrBuilder {
     // Use MultiRowRangeFilter.newBuilder() to construct.
-    private MultiRowRangeFilter(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MultiRowRangeFilter(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MultiRowRangeFilter() {
@@ -17621,18 +17621,18 @@ public final class FilterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MultiRowRangeFilter(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -17659,10 +17659,10 @@ public final class FilterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -17672,12 +17672,12 @@ public final class FilterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultiRowRangeFilter_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultiRowRangeFilter_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -17729,7 +17729,7 @@ public final class FilterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < rowRangeList_.size(); i++) {
         output.writeMessage(1, rowRangeList_.get(i));
@@ -17743,7 +17743,7 @@ public final class FilterProtos {
 
       size = 0;
       for (int i = 0; i < rowRangeList_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, rowRangeList_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -17786,61 +17786,61 @@ public final class FilterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -17858,7 +17858,7 @@ public final class FilterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -17866,15 +17866,15 @@ public final class FilterProtos {
      * Protobuf type {@code hbase.pb.MultiRowRangeFilter}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MultiRowRangeFilter)
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultiRowRangeFilter_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultiRowRangeFilter_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -17887,12 +17887,12 @@ public final class FilterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRowRangeListFieldBuilder();
         }
@@ -17908,7 +17908,7 @@ public final class FilterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.internal_static_hbase_pb_MultiRowRangeFilter_descriptor;
       }
@@ -17945,29 +17945,29 @@ public final class FilterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter)other);
         } else {
@@ -17997,7 +17997,7 @@ public final class FilterProtos {
               rowRangeList_ = other.rowRangeList_;
               bitField0_ = (bitField0_ & ~0x00000001);
               rowRangeListBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRowRangeListFieldBuilder() : null;
             } else {
               rowRangeListBuilder_.addAllMessages(other.rowRangeList_);
@@ -18014,13 +18014,13 @@ public final class FilterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.MultiRowRangeFilter) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -18041,7 +18041,7 @@ public final class FilterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRangeOrBuilder> rowRangeListBuilder_;
 
       /**
@@ -18173,7 +18173,7 @@ public final class FilterProtos {
           java.lang.Iterable values) {
         if (rowRangeListBuilder_ == null) {
           ensureRowRangeListIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, rowRangeList_);
           onChanged();
         } else {
@@ -18257,11 +18257,11 @@ public final class FilterProtos {
            getRowRangeListBuilderList() {
         return getRowRangeListFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRangeOrBuilder> 
           getRowRangeListFieldBuilder() {
         if (rowRangeListBuilder_ == null) {
-          rowRangeListBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          rowRangeListBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos.RowRangeOrBuilder>(
                   rowRangeList_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -18272,12 +18272,12 @@ public final class FilterProtos {
         return rowRangeListBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -18295,22 +18295,22 @@ public final class FilterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MultiRowRangeFilter parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MultiRowRangeFilter(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -18320,162 +18320,162 @@ public final class FilterProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Filter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Filter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ColumnCountGetFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ColumnCountGetFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ColumnPaginationFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ColumnPaginationFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ColumnPrefixFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ColumnPrefixFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ColumnRangeFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ColumnRangeFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CompareFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CompareFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DependentColumnFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DependentColumnFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FamilyFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FamilyFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FilterList_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FilterList_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FilterWrapper_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FilterWrapper_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FirstKeyOnlyFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FirstKeyOnlyFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FuzzyRowFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FuzzyRowFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_InclusiveStopFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_InclusiveStopFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_KeyOnlyFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_KeyOnlyFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MultipleColumnPrefixFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MultipleColumnPrefixFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_PageFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_PageFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_PrefixFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_PrefixFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_QualifierFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_QualifierFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RandomRowFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RandomRowFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RowFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RowFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SingleColumnValueExcludeFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SingleColumnValueExcludeFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SingleColumnValueFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SingleColumnValueFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SkipFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SkipFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimestampsFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TimestampsFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ValueFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ValueFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WhileMatchFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WhileMatchFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FilterAllFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FilterAllFilter_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RowRange_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RowRange_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MultiRowRangeFilter_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MultiRowRangeFilter_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -18537,198 +18537,198 @@ public final class FilterProtos {
       "ed.protobuf.generatedB\014FilterProtosH\001\210\001\001" +
       "\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_Filter_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_Filter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Filter_descriptor,
         new java.lang.String[] { "Name", "SerializedFilter", });
     internal_static_hbase_pb_ColumnCountGetFilter_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_ColumnCountGetFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ColumnCountGetFilter_descriptor,
         new java.lang.String[] { "Limit", });
     internal_static_hbase_pb_ColumnPaginationFilter_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_ColumnPaginationFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ColumnPaginationFilter_descriptor,
         new java.lang.String[] { "Limit", "Offset", "ColumnOffset", });
     internal_static_hbase_pb_ColumnPrefixFilter_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_ColumnPrefixFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ColumnPrefixFilter_descriptor,
         new java.lang.String[] { "Prefix", });
     internal_static_hbase_pb_ColumnRangeFilter_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_ColumnRangeFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ColumnRangeFilter_descriptor,
         new java.lang.String[] { "MinColumn", "MinColumnInclusive", "MaxColumn", "MaxColumnInclusive", });
     internal_static_hbase_pb_CompareFilter_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_CompareFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CompareFilter_descriptor,
         new java.lang.String[] { "CompareOp", "Comparator", });
     internal_static_hbase_pb_DependentColumnFilter_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_DependentColumnFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DependentColumnFilter_descriptor,
         new java.lang.String[] { "CompareFilter", "ColumnFamily", "ColumnQualifier", "DropDependentColumn", });
     internal_static_hbase_pb_FamilyFilter_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_FamilyFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FamilyFilter_descriptor,
         new java.lang.String[] { "CompareFilter", });
     internal_static_hbase_pb_FilterList_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_FilterList_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FilterList_descriptor,
         new java.lang.String[] { "Operator", "Filters", });
     internal_static_hbase_pb_FilterWrapper_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_FilterWrapper_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FilterWrapper_descriptor,
         new java.lang.String[] { "Filter", });
     internal_static_hbase_pb_FirstKeyOnlyFilter_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_FirstKeyOnlyFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FirstKeyOnlyFilter_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_descriptor =
       getDescriptor().getMessageTypes().get(11);
     internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FirstKeyValueMatchingQualifiersFilter_descriptor,
         new java.lang.String[] { "Qualifiers", });
     internal_static_hbase_pb_FuzzyRowFilter_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_FuzzyRowFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FuzzyRowFilter_descriptor,
         new java.lang.String[] { "FuzzyKeysData", });
     internal_static_hbase_pb_InclusiveStopFilter_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_hbase_pb_InclusiveStopFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_InclusiveStopFilter_descriptor,
         new java.lang.String[] { "StopRowKey", });
     internal_static_hbase_pb_KeyOnlyFilter_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_KeyOnlyFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_KeyOnlyFilter_descriptor,
         new java.lang.String[] { "LenAsVal", });
     internal_static_hbase_pb_MultipleColumnPrefixFilter_descriptor =
       getDescriptor().getMessageTypes().get(15);
     internal_static_hbase_pb_MultipleColumnPrefixFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MultipleColumnPrefixFilter_descriptor,
         new java.lang.String[] { "SortedPrefixes", });
     internal_static_hbase_pb_PageFilter_descriptor =
       getDescriptor().getMessageTypes().get(16);
     internal_static_hbase_pb_PageFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_PageFilter_descriptor,
         new java.lang.String[] { "PageSize", });
     internal_static_hbase_pb_PrefixFilter_descriptor =
       getDescriptor().getMessageTypes().get(17);
     internal_static_hbase_pb_PrefixFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_PrefixFilter_descriptor,
         new java.lang.String[] { "Prefix", });
     internal_static_hbase_pb_QualifierFilter_descriptor =
       getDescriptor().getMessageTypes().get(18);
     internal_static_hbase_pb_QualifierFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_QualifierFilter_descriptor,
         new java.lang.String[] { "CompareFilter", });
     internal_static_hbase_pb_RandomRowFilter_descriptor =
       getDescriptor().getMessageTypes().get(19);
     internal_static_hbase_pb_RandomRowFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RandomRowFilter_descriptor,
         new java.lang.String[] { "Chance", });
     internal_static_hbase_pb_RowFilter_descriptor =
       getDescriptor().getMessageTypes().get(20);
     internal_static_hbase_pb_RowFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RowFilter_descriptor,
         new java.lang.String[] { "CompareFilter", });
     internal_static_hbase_pb_SingleColumnValueExcludeFilter_descriptor =
       getDescriptor().getMessageTypes().get(21);
     internal_static_hbase_pb_SingleColumnValueExcludeFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SingleColumnValueExcludeFilter_descriptor,
         new java.lang.String[] { "SingleColumnValueFilter", });
     internal_static_hbase_pb_SingleColumnValueFilter_descriptor =
       getDescriptor().getMessageTypes().get(22);
     internal_static_hbase_pb_SingleColumnValueFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SingleColumnValueFilter_descriptor,
         new java.lang.String[] { "ColumnFamily", "ColumnQualifier", "CompareOp", "Comparator", "FilterIfMissing", "LatestVersionOnly", });
     internal_static_hbase_pb_SkipFilter_descriptor =
       getDescriptor().getMessageTypes().get(23);
     internal_static_hbase_pb_SkipFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SkipFilter_descriptor,
         new java.lang.String[] { "Filter", });
     internal_static_hbase_pb_TimestampsFilter_descriptor =
       getDescriptor().getMessageTypes().get(24);
     internal_static_hbase_pb_TimestampsFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TimestampsFilter_descriptor,
         new java.lang.String[] { "Timestamps", "CanHint", });
     internal_static_hbase_pb_ValueFilter_descriptor =
       getDescriptor().getMessageTypes().get(25);
     internal_static_hbase_pb_ValueFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ValueFilter_descriptor,
         new java.lang.String[] { "CompareFilter", });
     internal_static_hbase_pb_WhileMatchFilter_descriptor =
       getDescriptor().getMessageTypes().get(26);
     internal_static_hbase_pb_WhileMatchFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WhileMatchFilter_descriptor,
         new java.lang.String[] { "Filter", });
     internal_static_hbase_pb_FilterAllFilter_descriptor =
       getDescriptor().getMessageTypes().get(27);
     internal_static_hbase_pb_FilterAllFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FilterAllFilter_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_RowRange_descriptor =
       getDescriptor().getMessageTypes().get(28);
     internal_static_hbase_pb_RowRange_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RowRange_descriptor,
         new java.lang.String[] { "StartRow", "StartRowInclusive", "StopRow", "StopRowInclusive", });
     internal_static_hbase_pb_MultiRowRangeFilter_descriptor =
       getDescriptor().getMessageTypes().get(29);
     internal_static_hbase_pb_MultiRowRangeFilter_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MultiRowRangeFilter_descriptor,
         new java.lang.String[] { "RowRangeList", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
index 2ae9f9c..0617426 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
@@ -6,13 +6,13 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class HBaseProtos {
   private HBaseProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * 
@@ -22,7 +22,7 @@ public final class HBaseProtos {
    * Protobuf enum {@code hbase.pb.CompareType}
    */
   public enum CompareType
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * LESS = 0;
      */
@@ -108,27 +108,27 @@ public final class HBaseProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         CompareType> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public CompareType findValueByNumber(int number) {
               return CompareType.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -136,7 +136,7 @@ public final class HBaseProtos {
     private static final CompareType[] VALUES = values();
 
     public static CompareType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -157,7 +157,7 @@ public final class HBaseProtos {
    * Protobuf enum {@code hbase.pb.TimeUnit}
    */
   public enum TimeUnit
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * NANOSECONDS = 1;
      */
@@ -243,27 +243,27 @@ public final class HBaseProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         TimeUnit> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public TimeUnit findValueByNumber(int number) {
               return TimeUnit.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(1);
     }
@@ -271,7 +271,7 @@ public final class HBaseProtos {
     private static final TimeUnit[] VALUES = values();
 
     public static TimeUnit valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -290,7 +290,7 @@ public final class HBaseProtos {
 
   public interface TableNameOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TableName)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes namespace = 1;
@@ -299,7 +299,7 @@ public final class HBaseProtos {
     /**
      * required bytes namespace = 1;
      */
-    com.google.protobuf.ByteString getNamespace();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespace();
 
     /**
      * required bytes qualifier = 2;
@@ -308,7 +308,7 @@ public final class HBaseProtos {
     /**
      * required bytes qualifier = 2;
      */
-    com.google.protobuf.ByteString getQualifier();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier();
   }
   /**
    * 
@@ -319,31 +319,31 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.TableName}
    */
   public  static final class TableName extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TableName)
       TableNameOrBuilder {
     // Use TableName.newBuilder() to construct.
-    private TableName(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TableName(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TableName() {
-      namespace_ = com.google.protobuf.ByteString.EMPTY;
-      qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      namespace_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TableName(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -371,22 +371,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableName_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableName_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -395,7 +395,7 @@ public final class HBaseProtos {
 
     private int bitField0_;
     public static final int NAMESPACE_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString namespace_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString namespace_;
     /**
      * required bytes namespace = 1;
      */
@@ -405,12 +405,12 @@ public final class HBaseProtos {
     /**
      * required bytes namespace = 1;
      */
-    public com.google.protobuf.ByteString getNamespace() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespace() {
       return namespace_;
     }
 
     public static final int QUALIFIER_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString qualifier_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_;
     /**
      * required bytes qualifier = 2;
      */
@@ -420,7 +420,7 @@ public final class HBaseProtos {
     /**
      * required bytes qualifier = 2;
      */
-    public com.google.protobuf.ByteString getQualifier() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
       return qualifier_;
     }
 
@@ -442,7 +442,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, namespace_);
@@ -459,11 +459,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, namespace_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, qualifier_);
       }
       size += unknownFields.getSerializedSize();
@@ -518,61 +518,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -590,7 +590,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -603,15 +603,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.TableName}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TableName)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableName_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableName_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -624,25 +624,25 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        namespace_ = com.google.protobuf.ByteString.EMPTY;
+        namespace_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableName_descriptor;
       }
@@ -680,29 +680,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName)other);
         } else {
@@ -735,13 +735,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -753,7 +753,7 @@ public final class HBaseProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString namespace_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString namespace_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes namespace = 1;
        */
@@ -763,13 +763,13 @@ public final class HBaseProtos {
       /**
        * required bytes namespace = 1;
        */
-      public com.google.protobuf.ByteString getNamespace() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespace() {
         return namespace_;
       }
       /**
        * required bytes namespace = 1;
        */
-      public Builder setNamespace(com.google.protobuf.ByteString value) {
+      public Builder setNamespace(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -788,7 +788,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString qualifier_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes qualifier = 2;
        */
@@ -798,13 +798,13 @@ public final class HBaseProtos {
       /**
        * required bytes qualifier = 2;
        */
-      public com.google.protobuf.ByteString getQualifier() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getQualifier() {
         return qualifier_;
       }
       /**
        * required bytes qualifier = 2;
        */
-      public Builder setQualifier(com.google.protobuf.ByteString value) {
+      public Builder setQualifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -823,12 +823,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -846,22 +846,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TableName parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TableName(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -873,7 +873,7 @@ public final class HBaseProtos {
 
   public interface TableSchemaOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TableSchema)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.TableName table_name = 1;
@@ -970,11 +970,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.TableSchema}
    */
   public  static final class TableSchema extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TableSchema)
       TableSchemaOrBuilder {
     // Use TableSchema.newBuilder() to construct.
-    private TableSchema(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TableSchema(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TableSchema() {
@@ -984,18 +984,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TableSchema(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1053,10 +1053,10 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -1072,12 +1072,12 @@ public final class HBaseProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableSchema_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableSchema_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1245,7 +1245,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -1268,19 +1268,19 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       for (int i = 0; i < attributes_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, attributes_.get(i));
       }
       for (int i = 0; i < columnFamilies_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, columnFamilies_.get(i));
       }
       for (int i = 0; i < configuration_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, configuration_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -1344,61 +1344,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1416,7 +1416,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1430,15 +1430,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.TableSchema}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TableSchema)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableSchema_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableSchema_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1451,12 +1451,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getAttributesFieldBuilder();
@@ -1493,7 +1493,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableSchema_descriptor;
       }
@@ -1558,29 +1558,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema)other);
         } else {
@@ -1613,7 +1613,7 @@ public final class HBaseProtos {
               attributes_ = other.attributes_;
               bitField0_ = (bitField0_ & ~0x00000002);
               attributesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getAttributesFieldBuilder() : null;
             } else {
               attributesBuilder_.addAllMessages(other.attributes_);
@@ -1639,7 +1639,7 @@ public final class HBaseProtos {
               columnFamilies_ = other.columnFamilies_;
               bitField0_ = (bitField0_ & ~0x00000004);
               columnFamiliesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getColumnFamiliesFieldBuilder() : null;
             } else {
               columnFamiliesBuilder_.addAllMessages(other.columnFamilies_);
@@ -1665,7 +1665,7 @@ public final class HBaseProtos {
               configuration_ = other.configuration_;
               bitField0_ = (bitField0_ & ~0x00000008);
               configurationBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getConfigurationFieldBuilder() : null;
             } else {
               configurationBuilder_.addAllMessages(other.configuration_);
@@ -1702,13 +1702,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1721,7 +1721,7 @@ public final class HBaseProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * optional .hbase.pb.TableName table_name = 1;
@@ -1824,11 +1824,11 @@ public final class HBaseProtos {
       /**
        * optional .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -1847,7 +1847,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> attributesBuilder_;
 
       /**
@@ -1979,7 +1979,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (attributesBuilder_ == null) {
           ensureAttributesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, attributes_);
           onChanged();
         } else {
@@ -2063,11 +2063,11 @@ public final class HBaseProtos {
            getAttributesBuilderList() {
         return getAttributesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
           getAttributesFieldBuilder() {
         if (attributesBuilder_ == null) {
-          attributesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          attributesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>(
                   attributes_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -2087,7 +2087,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnFamiliesBuilder_;
 
       /**
@@ -2219,7 +2219,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (columnFamiliesBuilder_ == null) {
           ensureColumnFamiliesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, columnFamilies_);
           onChanged();
         } else {
@@ -2303,11 +2303,11 @@ public final class HBaseProtos {
            getColumnFamiliesBuilderList() {
         return getColumnFamiliesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
           getColumnFamiliesFieldBuilder() {
         if (columnFamiliesBuilder_ == null) {
-          columnFamiliesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          columnFamiliesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
                   columnFamilies_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -2327,7 +2327,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> configurationBuilder_;
 
       /**
@@ -2459,7 +2459,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (configurationBuilder_ == null) {
           ensureConfigurationIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, configuration_);
           onChanged();
         } else {
@@ -2543,11 +2543,11 @@ public final class HBaseProtos {
            getConfigurationBuilderList() {
         return getConfigurationFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
           getConfigurationFieldBuilder() {
         if (configurationBuilder_ == null) {
-          configurationBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          configurationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder>(
                   configuration_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -2558,12 +2558,12 @@ public final class HBaseProtos {
         return configurationBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2581,22 +2581,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TableSchema parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TableSchema(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2608,7 +2608,7 @@ public final class HBaseProtos {
 
   public interface TableStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TableState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -2635,11 +2635,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.TableState}
    */
   public  static final class TableState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TableState)
       TableStateOrBuilder {
     // Use TableState.newBuilder() to construct.
-    private TableState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TableState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TableState() {
@@ -2647,18 +2647,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TableState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2687,22 +2687,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2717,7 +2717,7 @@ public final class HBaseProtos {
      * Protobuf enum {@code hbase.pb.TableState.State}
      */
     public enum State
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * ENABLED = 0;
        */
@@ -2776,27 +2776,27 @@ public final class HBaseProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           State> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public State findValueByNumber(int number) {
                 return State.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState.getDescriptor().getEnumTypes().get(0);
       }
@@ -2804,7 +2804,7 @@ public final class HBaseProtos {
       private static final State[] VALUES = values();
 
       public static State valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -2860,7 +2860,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, state_);
@@ -2874,7 +2874,7 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, state_);
       }
       size += unknownFields.getSerializedSize();
@@ -2919,61 +2919,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2991,7 +2991,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3003,15 +3003,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.TableState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TableState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3024,12 +3024,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3040,7 +3040,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TableState_descriptor;
       }
@@ -3074,29 +3074,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState)other);
         } else {
@@ -3123,13 +3123,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3193,12 +3193,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3216,22 +3216,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TableState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TableState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3243,7 +3243,7 @@ public final class HBaseProtos {
 
   public interface ColumnFamilySchemaOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ColumnFamilySchema)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes name = 1;
@@ -3252,7 +3252,7 @@ public final class HBaseProtos {
     /**
      * required bytes name = 1;
      */
-    com.google.protobuf.ByteString getName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getName();
 
     /**
      * repeated .hbase.pb.BytesBytesPair attributes = 2;
@@ -3312,32 +3312,32 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.ColumnFamilySchema}
    */
   public  static final class ColumnFamilySchema extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ColumnFamilySchema)
       ColumnFamilySchemaOrBuilder {
     // Use ColumnFamilySchema.newBuilder() to construct.
-    private ColumnFamilySchema(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ColumnFamilySchema(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ColumnFamilySchema() {
-      name_ = com.google.protobuf.ByteString.EMPTY;
+      name_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       attributes_ = java.util.Collections.emptyList();
       configuration_ = java.util.Collections.emptyList();
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ColumnFamilySchema(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3378,10 +3378,10 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -3394,12 +3394,12 @@ public final class HBaseProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilySchema_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilySchema_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3408,7 +3408,7 @@ public final class HBaseProtos {
 
     private int bitField0_;
     public static final int NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString name_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString name_;
     /**
      * required bytes name = 1;
      */
@@ -3418,7 +3418,7 @@ public final class HBaseProtos {
     /**
      * required bytes name = 1;
      */
-    public com.google.protobuf.ByteString getName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getName() {
       return name_;
     }
 
@@ -3518,7 +3518,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, name_);
@@ -3538,15 +3538,15 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, name_);
       }
       for (int i = 0; i < attributes_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, attributes_.get(i));
       }
       for (int i = 0; i < configuration_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, configuration_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -3604,61 +3604,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3676,7 +3676,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3690,15 +3690,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.ColumnFamilySchema}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ColumnFamilySchema)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilySchema_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilySchema_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3711,12 +3711,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getAttributesFieldBuilder();
           getConfigurationFieldBuilder();
@@ -3724,7 +3724,7 @@ public final class HBaseProtos {
       }
       public Builder clear() {
         super.clear();
-        name_ = com.google.protobuf.ByteString.EMPTY;
+        name_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (attributesBuilder_ == null) {
           attributes_ = java.util.Collections.emptyList();
@@ -3741,7 +3741,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilySchema_descriptor;
       }
@@ -3793,29 +3793,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema)other);
         } else {
@@ -3848,7 +3848,7 @@ public final class HBaseProtos {
               attributes_ = other.attributes_;
               bitField0_ = (bitField0_ & ~0x00000002);
               attributesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getAttributesFieldBuilder() : null;
             } else {
               attributesBuilder_.addAllMessages(other.attributes_);
@@ -3874,7 +3874,7 @@ public final class HBaseProtos {
               configuration_ = other.configuration_;
               bitField0_ = (bitField0_ & ~0x00000004);
               configurationBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getConfigurationFieldBuilder() : null;
             } else {
               configurationBuilder_.addAllMessages(other.configuration_);
@@ -3904,13 +3904,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3922,7 +3922,7 @@ public final class HBaseProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString name_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString name_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes name = 1;
        */
@@ -3932,13 +3932,13 @@ public final class HBaseProtos {
       /**
        * required bytes name = 1;
        */
-      public com.google.protobuf.ByteString getName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getName() {
         return name_;
       }
       /**
        * required bytes name = 1;
        */
-      public Builder setName(com.google.protobuf.ByteString value) {
+      public Builder setName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3966,7 +3966,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> attributesBuilder_;
 
       /**
@@ -4098,7 +4098,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (attributesBuilder_ == null) {
           ensureAttributesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, attributes_);
           onChanged();
         } else {
@@ -4182,11 +4182,11 @@ public final class HBaseProtos {
            getAttributesBuilderList() {
         return getAttributesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
           getAttributesFieldBuilder() {
         if (attributesBuilder_ == null) {
-          attributesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          attributesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>(
                   attributes_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -4206,7 +4206,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> configurationBuilder_;
 
       /**
@@ -4338,7 +4338,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (configurationBuilder_ == null) {
           ensureConfigurationIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, configuration_);
           onChanged();
         } else {
@@ -4422,11 +4422,11 @@ public final class HBaseProtos {
            getConfigurationBuilderList() {
         return getConfigurationFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
           getConfigurationFieldBuilder() {
         if (configurationBuilder_ == null) {
-          configurationBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          configurationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder>(
                   configuration_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -4437,12 +4437,12 @@ public final class HBaseProtos {
         return configurationBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4460,22 +4460,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ColumnFamilySchema parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ColumnFamilySchema(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4487,7 +4487,7 @@ public final class HBaseProtos {
 
   public interface RegionInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 region_id = 1;
@@ -4518,7 +4518,7 @@ public final class HBaseProtos {
     /**
      * optional bytes start_key = 3;
      */
-    com.google.protobuf.ByteString getStartKey();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartKey();
 
     /**
      * optional bytes end_key = 4;
@@ -4527,7 +4527,7 @@ public final class HBaseProtos {
     /**
      * optional bytes end_key = 4;
      */
-    com.google.protobuf.ByteString getEndKey();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEndKey();
 
     /**
      * optional bool offline = 5;
@@ -4565,35 +4565,35 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.RegionInfo}
    */
   public  static final class RegionInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionInfo)
       RegionInfoOrBuilder {
     // Use RegionInfo.newBuilder() to construct.
-    private RegionInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionInfo() {
       regionId_ = 0L;
-      startKey_ = com.google.protobuf.ByteString.EMPTY;
-      endKey_ = com.google.protobuf.ByteString.EMPTY;
+      startKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      endKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       offline_ = false;
       split_ = false;
       replicaId_ = 0;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4654,22 +4654,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4714,7 +4714,7 @@ public final class HBaseProtos {
     }
 
     public static final int START_KEY_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString startKey_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString startKey_;
     /**
      * optional bytes start_key = 3;
      */
@@ -4724,12 +4724,12 @@ public final class HBaseProtos {
     /**
      * optional bytes start_key = 3;
      */
-    public com.google.protobuf.ByteString getStartKey() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartKey() {
       return startKey_;
     }
 
     public static final int END_KEY_FIELD_NUMBER = 4;
-    private com.google.protobuf.ByteString endKey_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString endKey_;
     /**
      * optional bytes end_key = 4;
      */
@@ -4739,7 +4739,7 @@ public final class HBaseProtos {
     /**
      * optional bytes end_key = 4;
      */
-    public com.google.protobuf.ByteString getEndKey() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEndKey() {
       return endKey_;
     }
 
@@ -4810,7 +4810,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, regionId_);
@@ -4842,31 +4842,31 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, regionId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, startKey_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(4, endKey_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, offline_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(6, split_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(7, replicaId_);
       }
       size += unknownFields.getSerializedSize();
@@ -4934,7 +4934,7 @@ public final class HBaseProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasRegionId()) {
         hash = (37 * hash) + REGION_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getRegionId());
       }
       if (hasTableName()) {
@@ -4951,12 +4951,12 @@ public final class HBaseProtos {
       }
       if (hasOffline()) {
         hash = (37 * hash) + OFFLINE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getOffline());
       }
       if (hasSplit()) {
         hash = (37 * hash) + SPLIT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getSplit());
       }
       if (hasReplicaId()) {
@@ -4969,61 +4969,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5041,7 +5041,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5054,15 +5054,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.RegionInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5075,12 +5075,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -5095,9 +5095,9 @@ public final class HBaseProtos {
           tableNameBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
-        startKey_ = com.google.protobuf.ByteString.EMPTY;
+        startKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
-        endKey_ = com.google.protobuf.ByteString.EMPTY;
+        endKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
         offline_ = false;
         bitField0_ = (bitField0_ & ~0x00000010);
@@ -5108,7 +5108,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionInfo_descriptor;
       }
@@ -5170,29 +5170,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo)other);
         } else {
@@ -5243,13 +5243,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5294,7 +5294,7 @@ public final class HBaseProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -5397,11 +5397,11 @@ public final class HBaseProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -5411,7 +5411,7 @@ public final class HBaseProtos {
         return tableNameBuilder_;
       }
 
-      private com.google.protobuf.ByteString startKey_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString startKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes start_key = 3;
        */
@@ -5421,13 +5421,13 @@ public final class HBaseProtos {
       /**
        * optional bytes start_key = 3;
        */
-      public com.google.protobuf.ByteString getStartKey() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStartKey() {
         return startKey_;
       }
       /**
        * optional bytes start_key = 3;
        */
-      public Builder setStartKey(com.google.protobuf.ByteString value) {
+      public Builder setStartKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5446,7 +5446,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString endKey_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString endKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes end_key = 4;
        */
@@ -5456,13 +5456,13 @@ public final class HBaseProtos {
       /**
        * optional bytes end_key = 4;
        */
-      public com.google.protobuf.ByteString getEndKey() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEndKey() {
         return endKey_;
       }
       /**
        * optional bytes end_key = 4;
        */
-      public Builder setEndKey(com.google.protobuf.ByteString value) {
+      public Builder setEndKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5577,12 +5577,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5600,22 +5600,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5627,7 +5627,7 @@ public final class HBaseProtos {
 
   public interface FavoredNodesOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FavoredNodes)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.ServerName favored_node = 1;
@@ -5662,11 +5662,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.FavoredNodes}
    */
   public  static final class FavoredNodes extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FavoredNodes)
       FavoredNodesOrBuilder {
     // Use FavoredNodes.newBuilder() to construct.
-    private FavoredNodes(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FavoredNodes(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FavoredNodes() {
@@ -5674,18 +5674,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FavoredNodes(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5712,10 +5712,10 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -5725,12 +5725,12 @@ public final class HBaseProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodes_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodes_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5788,7 +5788,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < favoredNode_.size(); i++) {
         output.writeMessage(1, favoredNode_.get(i));
@@ -5802,7 +5802,7 @@ public final class HBaseProtos {
 
       size = 0;
       for (int i = 0; i < favoredNode_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, favoredNode_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -5845,61 +5845,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5917,7 +5917,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5930,15 +5930,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.FavoredNodes}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FavoredNodes)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodes_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodes_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5951,12 +5951,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getFavoredNodeFieldBuilder();
         }
@@ -5972,7 +5972,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodes_descriptor;
       }
@@ -6009,29 +6009,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes)other);
         } else {
@@ -6061,7 +6061,7 @@ public final class HBaseProtos {
               favoredNode_ = other.favoredNode_;
               bitField0_ = (bitField0_ & ~0x00000001);
               favoredNodeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getFavoredNodeFieldBuilder() : null;
             } else {
               favoredNodeBuilder_.addAllMessages(other.favoredNode_);
@@ -6083,13 +6083,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6110,7 +6110,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> favoredNodeBuilder_;
 
       /**
@@ -6242,7 +6242,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (favoredNodeBuilder_ == null) {
           ensureFavoredNodeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, favoredNode_);
           onChanged();
         } else {
@@ -6326,11 +6326,11 @@ public final class HBaseProtos {
            getFavoredNodeBuilderList() {
         return getFavoredNodeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getFavoredNodeFieldBuilder() {
         if (favoredNodeBuilder_ == null) {
-          favoredNodeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          favoredNodeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   favoredNode_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -6341,12 +6341,12 @@ public final class HBaseProtos {
         return favoredNodeBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6364,22 +6364,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FavoredNodes parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FavoredNodes(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6391,7 +6391,7 @@ public final class HBaseProtos {
 
   public interface RegionSpecifierOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionSpecifier)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier.RegionSpecifierType type = 1;
@@ -6409,7 +6409,7 @@ public final class HBaseProtos {
     /**
      * required bytes value = 2;
      */
-    com.google.protobuf.ByteString getValue();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
   }
   /**
    * 
@@ -6423,31 +6423,31 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.RegionSpecifier}
    */
   public  static final class RegionSpecifier extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionSpecifier)
       RegionSpecifierOrBuilder {
     // Use RegionSpecifier.newBuilder() to construct.
-    private RegionSpecifier(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionSpecifier(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionSpecifier() {
       type_ = 1;
-      value_ = com.google.protobuf.ByteString.EMPTY;
+      value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionSpecifier(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6481,22 +6481,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionSpecifier_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionSpecifier_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6507,7 +6507,7 @@ public final class HBaseProtos {
      * Protobuf enum {@code hbase.pb.RegionSpecifier.RegionSpecifierType}
      */
     public enum RegionSpecifierType
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * 
        * <tablename>,<startkey>,<regionId>.<encodedName>
@@ -6564,27 +6564,27 @@ public final class HBaseProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           RegionSpecifierType> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public RegionSpecifierType findValueByNumber(int number) {
                 return RegionSpecifierType.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDescriptor().getEnumTypes().get(0);
       }
@@ -6592,7 +6592,7 @@ public final class HBaseProtos {
       private static final RegionSpecifierType[] VALUES = values();
 
       public static RegionSpecifierType valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -6627,7 +6627,7 @@ public final class HBaseProtos {
     }
 
     public static final int VALUE_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString value_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_;
     /**
      * required bytes value = 2;
      */
@@ -6637,7 +6637,7 @@ public final class HBaseProtos {
     /**
      * required bytes value = 2;
      */
-    public com.google.protobuf.ByteString getValue() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
@@ -6659,7 +6659,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, type_);
@@ -6676,11 +6676,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, type_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, value_);
       }
       size += unknownFields.getSerializedSize();
@@ -6734,61 +6734,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6806,7 +6806,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6822,15 +6822,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.RegionSpecifier}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionSpecifier)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionSpecifier_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionSpecifier_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6843,12 +6843,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -6856,12 +6856,12 @@ public final class HBaseProtos {
         super.clear();
         type_ = 1;
         bitField0_ = (bitField0_ & ~0x00000001);
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionSpecifier_descriptor;
       }
@@ -6899,29 +6899,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier)other);
         } else {
@@ -6954,13 +6954,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7008,7 +7008,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes value = 2;
        */
@@ -7018,13 +7018,13 @@ public final class HBaseProtos {
       /**
        * required bytes value = 2;
        */
-      public com.google.protobuf.ByteString getValue() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
         return value_;
       }
       /**
        * required bytes value = 2;
        */
-      public Builder setValue(com.google.protobuf.ByteString value) {
+      public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -7043,12 +7043,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7066,22 +7066,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionSpecifier parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionSpecifier(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7093,7 +7093,7 @@ public final class HBaseProtos {
 
   public interface TimeRangeOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TimeRange)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 from = 1;
@@ -7125,11 +7125,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.TimeRange}
    */
   public  static final class TimeRange extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TimeRange)
       TimeRangeOrBuilder {
     // Use TimeRange.newBuilder() to construct.
-    private TimeRange(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TimeRange(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TimeRange() {
@@ -7138,18 +7138,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TimeRange(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7177,22 +7177,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TimeRange_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TimeRange_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7240,7 +7240,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, from_);
@@ -7257,11 +7257,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, from_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, to_);
       }
       size += unknownFields.getSerializedSize();
@@ -7304,12 +7304,12 @@ public final class HBaseProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasFrom()) {
         hash = (37 * hash) + FROM_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getFrom());
       }
       if (hasTo()) {
         hash = (37 * hash) + TO_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTo());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -7318,61 +7318,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7390,7 +7390,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7406,15 +7406,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.TimeRange}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TimeRange)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TimeRange_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TimeRange_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7427,12 +7427,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7445,7 +7445,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_TimeRange_descriptor;
       }
@@ -7483,29 +7483,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange)other);
         } else {
@@ -7532,13 +7532,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7614,12 +7614,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7637,22 +7637,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TimeRange parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TimeRange(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7664,7 +7664,7 @@ public final class HBaseProtos {
 
   public interface ColumnFamilyTimeRangeOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ColumnFamilyTimeRange)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes column_family = 1;
@@ -7673,7 +7673,7 @@ public final class HBaseProtos {
     /**
      * required bytes column_family = 1;
      */
-    com.google.protobuf.ByteString getColumnFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily();
 
     /**
      * required .hbase.pb.TimeRange time_range = 2;
@@ -7696,30 +7696,30 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.ColumnFamilyTimeRange}
    */
   public  static final class ColumnFamilyTimeRange extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ColumnFamilyTimeRange)
       ColumnFamilyTimeRangeOrBuilder {
     // Use ColumnFamilyTimeRange.newBuilder() to construct.
-    private ColumnFamilyTimeRange(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ColumnFamilyTimeRange(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ColumnFamilyTimeRange() {
-      columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+      columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ColumnFamilyTimeRange(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7755,22 +7755,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7779,7 +7779,7 @@ public final class HBaseProtos {
 
     private int bitField0_;
     public static final int COLUMN_FAMILY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString columnFamily_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnFamily_;
     /**
      * required bytes column_family = 1;
      */
@@ -7789,7 +7789,7 @@ public final class HBaseProtos {
     /**
      * required bytes column_family = 1;
      */
-    public com.google.protobuf.ByteString getColumnFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily() {
       return columnFamily_;
     }
 
@@ -7832,7 +7832,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, columnFamily_);
@@ -7849,11 +7849,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, columnFamily_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTimeRange());
       }
       size += unknownFields.getSerializedSize();
@@ -7908,61 +7908,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7980,7 +7980,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7992,15 +7992,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.ColumnFamilyTimeRange}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ColumnFamilyTimeRange)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8013,19 +8013,19 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTimeRangeFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+        columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (timeRangeBuilder_ == null) {
           timeRange_ = null;
@@ -8036,7 +8036,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor;
       }
@@ -8078,29 +8078,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange)other);
         } else {
@@ -8133,13 +8133,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8151,7 +8151,7 @@ public final class HBaseProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString columnFamily_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnFamily_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes column_family = 1;
        */
@@ -8161,13 +8161,13 @@ public final class HBaseProtos {
       /**
        * required bytes column_family = 1;
        */
-      public com.google.protobuf.ByteString getColumnFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnFamily() {
         return columnFamily_;
       }
       /**
        * required bytes column_family = 1;
        */
-      public Builder setColumnFamily(com.google.protobuf.ByteString value) {
+      public Builder setColumnFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8187,7 +8187,7 @@ public final class HBaseProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange timeRange_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> timeRangeBuilder_;
       /**
        * required .hbase.pb.TimeRange time_range = 2;
@@ -8290,11 +8290,11 @@ public final class HBaseProtos {
       /**
        * required .hbase.pb.TimeRange time_range = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder> 
           getTimeRangeFieldBuilder() {
         if (timeRangeBuilder_ == null) {
-          timeRangeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          timeRangeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRange.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TimeRangeOrBuilder>(
                   getTimeRange(),
                   getParentForChildren(),
@@ -8304,12 +8304,12 @@ public final class HBaseProtos {
         return timeRangeBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8327,22 +8327,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ColumnFamilyTimeRange parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ColumnFamilyTimeRange(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8354,7 +8354,7 @@ public final class HBaseProtos {
 
   public interface ServerNameOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ServerName)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string host_name = 1;
@@ -8367,7 +8367,7 @@ public final class HBaseProtos {
     /**
      * required string host_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getHostNameBytes();
 
     /**
@@ -8397,11 +8397,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.ServerName}
    */
   public  static final class ServerName extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ServerName)
       ServerNameOrBuilder {
     // Use ServerName.newBuilder() to construct.
-    private ServerName(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ServerName(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ServerName() {
@@ -8411,18 +8411,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ServerName(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8439,7 +8439,7 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               hostName_ = bs;
               break;
@@ -8456,22 +8456,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerName_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerName_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8495,8 +8495,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           hostName_ = s;
@@ -8507,17 +8507,17 @@ public final class HBaseProtos {
     /**
      * required string host_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getHostNameBytes() {
       java.lang.Object ref = hostName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         hostName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -8565,10 +8565,10 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, hostName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, hostName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt32(2, port_);
@@ -8585,14 +8585,14 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, hostName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, hostName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, port_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, startCode_);
       }
       size += unknownFields.getSerializedSize();
@@ -8648,7 +8648,7 @@ public final class HBaseProtos {
       }
       if (hasStartCode()) {
         hash = (37 * hash) + START_CODE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getStartCode());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -8657,61 +8657,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8729,7 +8729,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8742,15 +8742,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.ServerName}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ServerName)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerName_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerName_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8763,12 +8763,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -8783,7 +8783,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerName_descriptor;
       }
@@ -8825,29 +8825,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName)other);
         } else {
@@ -8882,13 +8882,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8913,8 +8913,8 @@ public final class HBaseProtos {
       public java.lang.String getHostName() {
         java.lang.Object ref = hostName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             hostName_ = s;
@@ -8927,17 +8927,17 @@ public final class HBaseProtos {
       /**
        * required string host_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getHostNameBytes() {
         java.lang.Object ref = hostName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           hostName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -8966,7 +8966,7 @@ public final class HBaseProtos {
        * required string host_name = 1;
        */
       public Builder setHostNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -9040,12 +9040,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9063,22 +9063,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ServerName parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ServerName(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9090,7 +9090,7 @@ public final class HBaseProtos {
 
   public interface CoprocessorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Coprocessor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string name = 1;
@@ -9103,18 +9103,18 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.Coprocessor}
    */
   public  static final class Coprocessor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Coprocessor)
       CoprocessorOrBuilder {
     // Use Coprocessor.newBuilder() to construct.
-    private Coprocessor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Coprocessor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Coprocessor() {
@@ -9122,18 +9122,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Coprocessor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9150,29 +9150,29 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_Coprocessor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_Coprocessor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9196,8 +9196,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -9208,17 +9208,17 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -9236,10 +9236,10 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       unknownFields.writeTo(output);
     }
@@ -9250,7 +9250,7 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -9295,61 +9295,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9367,7 +9367,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9375,15 +9375,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.Coprocessor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Coprocessor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CoprocessorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_Coprocessor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_Coprocessor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9396,12 +9396,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -9412,7 +9412,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_Coprocessor_descriptor;
       }
@@ -9446,29 +9446,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor)other);
         } else {
@@ -9497,13 +9497,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9528,8 +9528,8 @@ public final class HBaseProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -9542,17 +9542,17 @@ public final class HBaseProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -9581,7 +9581,7 @@ public final class HBaseProtos {
        * required string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -9591,12 +9591,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9614,22 +9614,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Coprocessor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Coprocessor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9641,7 +9641,7 @@ public final class HBaseProtos {
 
   public interface NameStringPairOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NameStringPair)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string name = 1;
@@ -9654,7 +9654,7 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
 
     /**
@@ -9668,18 +9668,18 @@ public final class HBaseProtos {
     /**
      * required string value = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getValueBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.NameStringPair}
    */
   public  static final class NameStringPair extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NameStringPair)
       NameStringPairOrBuilder {
     // Use NameStringPair.newBuilder() to construct.
-    private NameStringPair(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NameStringPair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NameStringPair() {
@@ -9688,18 +9688,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NameStringPair(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9716,35 +9716,35 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               value_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameStringPair_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameStringPair_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9768,8 +9768,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -9780,17 +9780,17 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -9810,8 +9810,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           value_ = s;
@@ -9822,17 +9822,17 @@ public final class HBaseProtos {
     /**
      * required string value = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getValueBytes() {
       java.lang.Object ref = value_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         value_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -9854,13 +9854,13 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, value_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, value_);
       }
       unknownFields.writeTo(output);
     }
@@ -9871,10 +9871,10 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, value_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, value_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -9928,61 +9928,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10000,7 +10000,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10008,15 +10008,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.NameStringPair}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NameStringPair)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameStringPair_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameStringPair_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10029,12 +10029,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -10047,7 +10047,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameStringPair_descriptor;
       }
@@ -10085,29 +10085,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair)other);
         } else {
@@ -10144,13 +10144,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -10175,8 +10175,8 @@ public final class HBaseProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -10189,17 +10189,17 @@ public final class HBaseProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -10228,7 +10228,7 @@ public final class HBaseProtos {
        * required string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10251,8 +10251,8 @@ public final class HBaseProtos {
       public java.lang.String getValue() {
         java.lang.Object ref = value_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             value_ = s;
@@ -10265,17 +10265,17 @@ public final class HBaseProtos {
       /**
        * required string value = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getValueBytes() {
         java.lang.Object ref = value_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           value_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -10304,7 +10304,7 @@ public final class HBaseProtos {
        * required string value = 2;
        */
       public Builder setValueBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10314,12 +10314,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10337,22 +10337,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NameStringPair parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NameStringPair(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10364,7 +10364,7 @@ public final class HBaseProtos {
 
   public interface NameBytesPairOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NameBytesPair)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string name = 1;
@@ -10377,7 +10377,7 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
 
     /**
@@ -10387,37 +10387,37 @@ public final class HBaseProtos {
     /**
      * optional bytes value = 2;
      */
-    com.google.protobuf.ByteString getValue();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
   }
   /**
    * Protobuf type {@code hbase.pb.NameBytesPair}
    */
   public  static final class NameBytesPair extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NameBytesPair)
       NameBytesPairOrBuilder {
     // Use NameBytesPair.newBuilder() to construct.
-    private NameBytesPair(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NameBytesPair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NameBytesPair() {
       name_ = "";
-      value_ = com.google.protobuf.ByteString.EMPTY;
+      value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NameBytesPair(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10434,7 +10434,7 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
@@ -10446,22 +10446,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameBytesPair_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameBytesPair_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10485,8 +10485,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -10497,22 +10497,22 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int VALUE_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString value_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_;
     /**
      * optional bytes value = 2;
      */
@@ -10522,7 +10522,7 @@ public final class HBaseProtos {
     /**
      * optional bytes value = 2;
      */
-    public com.google.protobuf.ByteString getValue() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
     }
 
@@ -10540,10 +10540,10 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBytes(2, value_);
@@ -10557,10 +10557,10 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, value_);
       }
       size += unknownFields.getSerializedSize();
@@ -10615,61 +10615,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10687,7 +10687,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10695,15 +10695,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.NameBytesPair}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NameBytesPair)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameBytesPair_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameBytesPair_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10716,12 +10716,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -10729,12 +10729,12 @@ public final class HBaseProtos {
         super.clear();
         name_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
-        value_ = com.google.protobuf.ByteString.EMPTY;
+        value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameBytesPair_descriptor;
       }
@@ -10772,29 +10772,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair)other);
         } else {
@@ -10826,13 +10826,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -10857,8 +10857,8 @@ public final class HBaseProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -10871,17 +10871,17 @@ public final class HBaseProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -10910,7 +10910,7 @@ public final class HBaseProtos {
        * required string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10920,7 +10920,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes value = 2;
        */
@@ -10930,13 +10930,13 @@ public final class HBaseProtos {
       /**
        * optional bytes value = 2;
        */
-      public com.google.protobuf.ByteString getValue() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
         return value_;
       }
       /**
        * optional bytes value = 2;
        */
-      public Builder setValue(com.google.protobuf.ByteString value) {
+      public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10955,12 +10955,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10978,22 +10978,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NameBytesPair parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NameBytesPair(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11005,7 +11005,7 @@ public final class HBaseProtos {
 
   public interface BytesBytesPairOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BytesBytesPair)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes first = 1;
@@ -11014,7 +11014,7 @@ public final class HBaseProtos {
     /**
      * required bytes first = 1;
      */
-    com.google.protobuf.ByteString getFirst();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFirst();
 
     /**
      * required bytes second = 2;
@@ -11023,37 +11023,37 @@ public final class HBaseProtos {
     /**
      * required bytes second = 2;
      */
-    com.google.protobuf.ByteString getSecond();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSecond();
   }
   /**
    * Protobuf type {@code hbase.pb.BytesBytesPair}
    */
   public  static final class BytesBytesPair extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BytesBytesPair)
       BytesBytesPairOrBuilder {
     // Use BytesBytesPair.newBuilder() to construct.
-    private BytesBytesPair(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BytesBytesPair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BytesBytesPair() {
-      first_ = com.google.protobuf.ByteString.EMPTY;
-      second_ = com.google.protobuf.ByteString.EMPTY;
+      first_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      second_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BytesBytesPair(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11081,22 +11081,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BytesBytesPair_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BytesBytesPair_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11105,7 +11105,7 @@ public final class HBaseProtos {
 
     private int bitField0_;
     public static final int FIRST_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString first_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString first_;
     /**
      * required bytes first = 1;
      */
@@ -11115,12 +11115,12 @@ public final class HBaseProtos {
     /**
      * required bytes first = 1;
      */
-    public com.google.protobuf.ByteString getFirst() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFirst() {
       return first_;
     }
 
     public static final int SECOND_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString second_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString second_;
     /**
      * required bytes second = 2;
      */
@@ -11130,7 +11130,7 @@ public final class HBaseProtos {
     /**
      * required bytes second = 2;
      */
-    public com.google.protobuf.ByteString getSecond() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSecond() {
       return second_;
     }
 
@@ -11152,7 +11152,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, first_);
@@ -11169,11 +11169,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, first_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, second_);
       }
       size += unknownFields.getSerializedSize();
@@ -11228,61 +11228,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11300,7 +11300,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11308,15 +11308,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.BytesBytesPair}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BytesBytesPair)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BytesBytesPair_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BytesBytesPair_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11329,25 +11329,25 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        first_ = com.google.protobuf.ByteString.EMPTY;
+        first_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        second_ = com.google.protobuf.ByteString.EMPTY;
+        second_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BytesBytesPair_descriptor;
       }
@@ -11385,29 +11385,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair)other);
         } else {
@@ -11440,13 +11440,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11458,7 +11458,7 @@ public final class HBaseProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString first_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString first_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes first = 1;
        */
@@ -11468,13 +11468,13 @@ public final class HBaseProtos {
       /**
        * required bytes first = 1;
        */
-      public com.google.protobuf.ByteString getFirst() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFirst() {
         return first_;
       }
       /**
        * required bytes first = 1;
        */
-      public Builder setFirst(com.google.protobuf.ByteString value) {
+      public Builder setFirst(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11493,7 +11493,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString second_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString second_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes second = 2;
        */
@@ -11503,13 +11503,13 @@ public final class HBaseProtos {
       /**
        * required bytes second = 2;
        */
-      public com.google.protobuf.ByteString getSecond() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSecond() {
         return second_;
       }
       /**
        * required bytes second = 2;
        */
-      public Builder setSecond(com.google.protobuf.ByteString value) {
+      public Builder setSecond(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11528,12 +11528,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11551,22 +11551,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BytesBytesPair parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BytesBytesPair(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11578,7 +11578,7 @@ public final class HBaseProtos {
 
   public interface NameInt64PairOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NameInt64Pair)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional string name = 1;
@@ -11591,7 +11591,7 @@ public final class HBaseProtos {
     /**
      * optional string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
 
     /**
@@ -11607,11 +11607,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.NameInt64Pair}
    */
   public  static final class NameInt64Pair extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NameInt64Pair)
       NameInt64PairOrBuilder {
     // Use NameInt64Pair.newBuilder() to construct.
-    private NameInt64Pair(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NameInt64Pair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NameInt64Pair() {
@@ -11620,18 +11620,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NameInt64Pair(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11648,7 +11648,7 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
@@ -11660,22 +11660,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameInt64Pair_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameInt64Pair_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11699,8 +11699,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -11711,17 +11711,17 @@ public final class HBaseProtos {
     /**
      * optional string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -11750,10 +11750,10 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt64(2, value_);
@@ -11767,10 +11767,10 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(2, value_);
       }
       size += unknownFields.getSerializedSize();
@@ -11817,7 +11817,7 @@ public final class HBaseProtos {
       }
       if (hasValue()) {
         hash = (37 * hash) + VALUE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getValue());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -11826,61 +11826,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11898,7 +11898,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11906,15 +11906,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.NameInt64Pair}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NameInt64Pair)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64PairOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameInt64Pair_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameInt64Pair_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11927,12 +11927,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -11945,7 +11945,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NameInt64Pair_descriptor;
       }
@@ -11983,29 +11983,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair)other);
         } else {
@@ -12034,13 +12034,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12065,8 +12065,8 @@ public final class HBaseProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -12079,17 +12079,17 @@ public final class HBaseProtos {
       /**
        * optional string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -12118,7 +12118,7 @@ public final class HBaseProtos {
        * optional string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -12160,12 +12160,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12183,22 +12183,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NameInt64Pair parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NameInt64Pair(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12210,7 +12210,7 @@ public final class HBaseProtos {
 
   public interface SnapshotDescriptionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotDescription)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string name = 1;
@@ -12223,7 +12223,7 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes();
 
     /**
@@ -12249,7 +12249,7 @@ public final class HBaseProtos {
      *
      * optional string table = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getTableBytes();
 
     /**
@@ -12290,7 +12290,7 @@ public final class HBaseProtos {
     /**
      * optional string owner = 6;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getOwnerBytes();
   }
   /**
@@ -12302,11 +12302,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.SnapshotDescription}
    */
   public  static final class SnapshotDescription extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotDescription)
       SnapshotDescriptionOrBuilder {
     // Use SnapshotDescription.newBuilder() to construct.
-    private SnapshotDescription(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SnapshotDescription(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SnapshotDescription() {
@@ -12319,18 +12319,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SnapshotDescription(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12347,13 +12347,13 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               name_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               table_ = bs;
               break;
@@ -12380,29 +12380,29 @@ public final class HBaseProtos {
               break;
             }
             case 50: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000020;
               owner_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12413,7 +12413,7 @@ public final class HBaseProtos {
      * Protobuf enum {@code hbase.pb.SnapshotDescription.Type}
      */
     public enum Type
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * DISABLED = 0;
        */
@@ -12463,27 +12463,27 @@ public final class HBaseProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Type> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Type findValueByNumber(int number) {
                 return Type.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.getDescriptor().getEnumTypes().get(0);
       }
@@ -12491,7 +12491,7 @@ public final class HBaseProtos {
       private static final Type[] VALUES = values();
 
       public static Type valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -12525,8 +12525,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           name_ = s;
@@ -12537,17 +12537,17 @@ public final class HBaseProtos {
     /**
      * required string name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -12575,8 +12575,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           table_ = s;
@@ -12591,17 +12591,17 @@ public final class HBaseProtos {
      *
      * optional string table = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getTableBytes() {
       java.lang.Object ref = table_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         table_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -12667,8 +12667,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           owner_ = s;
@@ -12679,17 +12679,17 @@ public final class HBaseProtos {
     /**
      * optional string owner = 6;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getOwnerBytes() {
       java.lang.Object ref = owner_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         owner_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -12707,13 +12707,13 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, table_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, table_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt64(3, creationTime_);
@@ -12725,7 +12725,7 @@ public final class HBaseProtos {
         output.writeInt32(5, version_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 6, owner_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 6, owner_);
       }
       unknownFields.writeTo(output);
     }
@@ -12736,25 +12736,25 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, table_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, table_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(3, creationTime_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(4, type_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(5, version_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, owner_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(6, owner_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -12823,7 +12823,7 @@ public final class HBaseProtos {
       }
       if (hasCreationTime()) {
         hash = (37 * hash) + CREATION_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getCreationTime());
       }
       if (hasType()) {
@@ -12844,61 +12844,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12916,7 +12916,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12929,15 +12929,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.SnapshotDescription}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotDescription)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12950,12 +12950,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -12976,7 +12976,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_SnapshotDescription_descriptor;
       }
@@ -13030,29 +13030,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription)other);
         } else {
@@ -13100,13 +13100,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13131,8 +13131,8 @@ public final class HBaseProtos {
       public java.lang.String getName() {
         java.lang.Object ref = name_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -13145,17 +13145,17 @@ public final class HBaseProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -13184,7 +13184,7 @@ public final class HBaseProtos {
        * required string name = 1;
        */
       public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -13215,8 +13215,8 @@ public final class HBaseProtos {
       public java.lang.String getTable() {
         java.lang.Object ref = table_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             table_ = s;
@@ -13233,17 +13233,17 @@ public final class HBaseProtos {
        *
        * optional string table = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getTableBytes() {
         java.lang.Object ref = table_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           table_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -13284,7 +13284,7 @@ public final class HBaseProtos {
        * optional string table = 2;
        */
       public Builder setTableBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -13407,8 +13407,8 @@ public final class HBaseProtos {
       public java.lang.String getOwner() {
         java.lang.Object ref = owner_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             owner_ = s;
@@ -13421,17 +13421,17 @@ public final class HBaseProtos {
       /**
        * optional string owner = 6;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getOwnerBytes() {
         java.lang.Object ref = owner_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           owner_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -13460,7 +13460,7 @@ public final class HBaseProtos {
        * optional string owner = 6;
        */
       public Builder setOwnerBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -13470,12 +13470,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -13493,22 +13493,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SnapshotDescription parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SnapshotDescription(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13520,7 +13520,7 @@ public final class HBaseProtos {
 
   public interface ProcedureDescriptionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ProcedureDescription)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -13545,7 +13545,7 @@ public final class HBaseProtos {
      *
      * required string signature = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSignatureBytes();
 
     /**
@@ -13571,7 +13571,7 @@ public final class HBaseProtos {
      *
      * optional string instance = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getInstanceBytes();
 
     /**
@@ -13616,11 +13616,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.ProcedureDescription}
    */
   public  static final class ProcedureDescription extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ProcedureDescription)
       ProcedureDescriptionOrBuilder {
     // Use ProcedureDescription.newBuilder() to construct.
-    private ProcedureDescription(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ProcedureDescription(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ProcedureDescription() {
@@ -13631,18 +13631,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ProcedureDescription(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -13659,13 +13659,13 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               signature_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               instance_ = bs;
               break;
@@ -13686,10 +13686,10 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
@@ -13699,12 +13699,12 @@ public final class HBaseProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ProcedureDescription_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ProcedureDescription_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -13736,8 +13736,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           signature_ = s;
@@ -13752,17 +13752,17 @@ public final class HBaseProtos {
      *
      * required string signature = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSignatureBytes() {
       java.lang.Object ref = signature_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         signature_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -13790,8 +13790,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           instance_ = s;
@@ -13806,17 +13806,17 @@ public final class HBaseProtos {
      *
      * optional string instance = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getInstanceBytes() {
       java.lang.Object ref = instance_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         instance_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -13890,13 +13890,13 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, signature_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, signature_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, instance_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, instance_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt64(3, creationTime_);
@@ -13913,17 +13913,17 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, signature_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, signature_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, instance_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, instance_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(3, creationTime_);
       }
       for (int i = 0; i < configuration_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, configuration_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -13981,7 +13981,7 @@ public final class HBaseProtos {
       }
       if (hasCreationTime()) {
         hash = (37 * hash) + CREATION_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getCreationTime());
       }
       if (getConfigurationCount() > 0) {
@@ -13994,61 +13994,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14066,7 +14066,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -14079,15 +14079,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.ProcedureDescription}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ProcedureDescription)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ProcedureDescription_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ProcedureDescription_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -14100,12 +14100,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getConfigurationFieldBuilder();
         }
@@ -14127,7 +14127,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ProcedureDescription_descriptor;
       }
@@ -14178,29 +14178,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription)other);
         } else {
@@ -14243,7 +14243,7 @@ public final class HBaseProtos {
               configuration_ = other.configuration_;
               bitField0_ = (bitField0_ & ~0x00000008);
               configurationBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getConfigurationFieldBuilder() : null;
             } else {
               configurationBuilder_.addAllMessages(other.configuration_);
@@ -14268,13 +14268,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -14307,8 +14307,8 @@ public final class HBaseProtos {
       public java.lang.String getSignature() {
         java.lang.Object ref = signature_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             signature_ = s;
@@ -14325,17 +14325,17 @@ public final class HBaseProtos {
        *
        * required string signature = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getSignatureBytes() {
         java.lang.Object ref = signature_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           signature_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -14376,7 +14376,7 @@ public final class HBaseProtos {
        * required string signature = 1;
        */
       public Builder setSignatureBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -14407,8 +14407,8 @@ public final class HBaseProtos {
       public java.lang.String getInstance() {
         java.lang.Object ref = instance_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             instance_ = s;
@@ -14425,17 +14425,17 @@ public final class HBaseProtos {
        *
        * optional string instance = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getInstanceBytes() {
         java.lang.Object ref = instance_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           instance_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -14476,7 +14476,7 @@ public final class HBaseProtos {
        * optional string instance = 2;
        */
       public Builder setInstanceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -14527,7 +14527,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> configurationBuilder_;
 
       /**
@@ -14659,7 +14659,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (configurationBuilder_ == null) {
           ensureConfigurationIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, configuration_);
           onChanged();
         } else {
@@ -14743,11 +14743,11 @@ public final class HBaseProtos {
            getConfigurationBuilderList() {
         return getConfigurationFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
           getConfigurationFieldBuilder() {
         if (configurationBuilder_ == null) {
-          configurationBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          configurationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder>(
                   configuration_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -14758,12 +14758,12 @@ public final class HBaseProtos {
         return configurationBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14781,22 +14781,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ProcedureDescription parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ProcedureDescription(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14808,34 +14808,34 @@ public final class HBaseProtos {
 
   public interface EmptyMsgOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.EmptyMsg)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.EmptyMsg}
    */
   public  static final class EmptyMsg extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.EmptyMsg)
       EmptyMsgOrBuilder {
     // Use EmptyMsg.newBuilder() to construct.
-    private EmptyMsg(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private EmptyMsg(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private EmptyMsg() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private EmptyMsg(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14853,22 +14853,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_EmptyMsg_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_EmptyMsg_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14885,7 +14885,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -14929,61 +14929,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15001,7 +15001,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15009,15 +15009,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.EmptyMsg}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.EmptyMsg)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsgOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_EmptyMsg_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_EmptyMsg_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15030,12 +15030,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -15044,7 +15044,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_EmptyMsg_descriptor;
       }
@@ -15071,29 +15071,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg)other);
         } else {
@@ -15114,13 +15114,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15131,12 +15131,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15154,22 +15154,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public EmptyMsg parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new EmptyMsg(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15181,7 +15181,7 @@ public final class HBaseProtos {
 
   public interface LongMsgOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.LongMsg)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int64 long_msg = 1;
@@ -15196,11 +15196,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.LongMsg}
    */
   public  static final class LongMsg extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.LongMsg)
       LongMsgOrBuilder {
     // Use LongMsg.newBuilder() to construct.
-    private LongMsg(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private LongMsg(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private LongMsg() {
@@ -15208,18 +15208,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private LongMsg(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15242,22 +15242,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LongMsg_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LongMsg_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -15294,7 +15294,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, longMsg_);
@@ -15308,7 +15308,7 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, longMsg_);
       }
       size += unknownFields.getSerializedSize();
@@ -15346,7 +15346,7 @@ public final class HBaseProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLongMsg()) {
         hash = (37 * hash) + LONG_MSG_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLongMsg());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -15355,61 +15355,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15427,7 +15427,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15435,15 +15435,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.LongMsg}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.LongMsg)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsgOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LongMsg_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LongMsg_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15456,12 +15456,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -15472,7 +15472,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_LongMsg_descriptor;
       }
@@ -15506,29 +15506,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg)other);
         } else {
@@ -15555,13 +15555,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15605,12 +15605,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15628,22 +15628,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public LongMsg parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new LongMsg(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15655,7 +15655,7 @@ public final class HBaseProtos {
 
   public interface DoubleMsgOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DoubleMsg)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required double double_msg = 1;
@@ -15670,11 +15670,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.DoubleMsg}
    */
   public  static final class DoubleMsg extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DoubleMsg)
       DoubleMsgOrBuilder {
     // Use DoubleMsg.newBuilder() to construct.
-    private DoubleMsg(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DoubleMsg(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DoubleMsg() {
@@ -15682,18 +15682,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DoubleMsg(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15716,22 +15716,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_DoubleMsg_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_DoubleMsg_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -15768,7 +15768,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeDouble(1, doubleMsg_);
@@ -15782,7 +15782,7 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeDoubleSize(1, doubleMsg_);
       }
       size += unknownFields.getSerializedSize();
@@ -15822,7 +15822,7 @@ public final class HBaseProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasDoubleMsg()) {
         hash = (37 * hash) + DOUBLE_MSG_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             java.lang.Double.doubleToLongBits(getDoubleMsg()));
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -15831,61 +15831,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15903,7 +15903,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15911,15 +15911,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.DoubleMsg}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DoubleMsg)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsgOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_DoubleMsg_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_DoubleMsg_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15932,12 +15932,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -15948,7 +15948,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_DoubleMsg_descriptor;
       }
@@ -15982,29 +15982,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg)other);
         } else {
@@ -16031,13 +16031,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16081,12 +16081,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16104,22 +16104,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DoubleMsg parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DoubleMsg(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16131,7 +16131,7 @@ public final class HBaseProtos {
 
   public interface BigDecimalMsgOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BigDecimalMsg)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes bigdecimal_msg = 1;
@@ -16140,36 +16140,36 @@ public final class HBaseProtos {
     /**
      * required bytes bigdecimal_msg = 1;
      */
-    com.google.protobuf.ByteString getBigdecimalMsg();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getBigdecimalMsg();
   }
   /**
    * Protobuf type {@code hbase.pb.BigDecimalMsg}
    */
   public  static final class BigDecimalMsg extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BigDecimalMsg)
       BigDecimalMsgOrBuilder {
     // Use BigDecimalMsg.newBuilder() to construct.
-    private BigDecimalMsg(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BigDecimalMsg(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BigDecimalMsg() {
-      bigdecimalMsg_ = com.google.protobuf.ByteString.EMPTY;
+      bigdecimalMsg_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BigDecimalMsg(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16192,22 +16192,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BigDecimalMsg_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BigDecimalMsg_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16216,7 +16216,7 @@ public final class HBaseProtos {
 
     private int bitField0_;
     public static final int BIGDECIMAL_MSG_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString bigdecimalMsg_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bigdecimalMsg_;
     /**
      * required bytes bigdecimal_msg = 1;
      */
@@ -16226,7 +16226,7 @@ public final class HBaseProtos {
     /**
      * required bytes bigdecimal_msg = 1;
      */
-    public com.google.protobuf.ByteString getBigdecimalMsg() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getBigdecimalMsg() {
       return bigdecimalMsg_;
     }
 
@@ -16244,7 +16244,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, bigdecimalMsg_);
@@ -16258,7 +16258,7 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, bigdecimalMsg_);
       }
       size += unknownFields.getSerializedSize();
@@ -16304,61 +16304,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16376,7 +16376,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16384,15 +16384,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.BigDecimalMsg}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BigDecimalMsg)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsgOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BigDecimalMsg_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BigDecimalMsg_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16405,23 +16405,23 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        bigdecimalMsg_ = com.google.protobuf.ByteString.EMPTY;
+        bigdecimalMsg_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_BigDecimalMsg_descriptor;
       }
@@ -16455,29 +16455,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg)other);
         } else {
@@ -16504,13 +16504,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16522,7 +16522,7 @@ public final class HBaseProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString bigdecimalMsg_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bigdecimalMsg_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes bigdecimal_msg = 1;
        */
@@ -16532,13 +16532,13 @@ public final class HBaseProtos {
       /**
        * required bytes bigdecimal_msg = 1;
        */
-      public com.google.protobuf.ByteString getBigdecimalMsg() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getBigdecimalMsg() {
         return bigdecimalMsg_;
       }
       /**
        * required bytes bigdecimal_msg = 1;
        */
-      public Builder setBigdecimalMsg(com.google.protobuf.ByteString value) {
+      public Builder setBigdecimalMsg(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -16557,12 +16557,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16580,22 +16580,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BigDecimalMsg parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BigDecimalMsg(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16607,7 +16607,7 @@ public final class HBaseProtos {
 
   public interface UUIDOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UUID)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 least_sig_bits = 1;
@@ -16631,11 +16631,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.UUID}
    */
   public  static final class UUID extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UUID)
       UUIDOrBuilder {
     // Use UUID.newBuilder() to construct.
-    private UUID(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UUID(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UUID() {
@@ -16644,18 +16644,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UUID(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16683,22 +16683,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_UUID_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_UUID_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16754,7 +16754,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, leastSigBits_);
@@ -16771,11 +16771,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, leastSigBits_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, mostSigBits_);
       }
       size += unknownFields.getSerializedSize();
@@ -16818,12 +16818,12 @@ public final class HBaseProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLeastSigBits()) {
         hash = (37 * hash) + LEAST_SIG_BITS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLeastSigBits());
       }
       if (hasMostSigBits()) {
         hash = (37 * hash) + MOST_SIG_BITS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getMostSigBits());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -16832,61 +16832,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16904,7 +16904,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16912,15 +16912,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.UUID}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UUID)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_UUID_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_UUID_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16933,12 +16933,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -16951,7 +16951,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_UUID_descriptor;
       }
@@ -16989,29 +16989,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID)other);
         } else {
@@ -17044,13 +17044,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -17126,12 +17126,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17149,22 +17149,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UUID parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UUID(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17176,7 +17176,7 @@ public final class HBaseProtos {
 
   public interface NamespaceDescriptorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NamespaceDescriptor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes name = 1;
@@ -17185,7 +17185,7 @@ public final class HBaseProtos {
     /**
      * required bytes name = 1;
      */
-    com.google.protobuf.ByteString getName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getName();
 
     /**
      * repeated .hbase.pb.NameStringPair configuration = 2;
@@ -17215,31 +17215,31 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.NamespaceDescriptor}
    */
   public  static final class NamespaceDescriptor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NamespaceDescriptor)
       NamespaceDescriptorOrBuilder {
     // Use NamespaceDescriptor.newBuilder() to construct.
-    private NamespaceDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NamespaceDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NamespaceDescriptor() {
-      name_ = com.google.protobuf.ByteString.EMPTY;
+      name_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       configuration_ = java.util.Collections.emptyList();
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NamespaceDescriptor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -17271,10 +17271,10 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -17284,12 +17284,12 @@ public final class HBaseProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NamespaceDescriptor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NamespaceDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -17298,7 +17298,7 @@ public final class HBaseProtos {
 
     private int bitField0_;
     public static final int NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString name_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString name_;
     /**
      * required bytes name = 1;
      */
@@ -17308,7 +17308,7 @@ public final class HBaseProtos {
     /**
      * required bytes name = 1;
      */
-    public com.google.protobuf.ByteString getName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getName() {
       return name_;
     }
 
@@ -17367,7 +17367,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, name_);
@@ -17384,11 +17384,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, name_);
       }
       for (int i = 0; i < configuration_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, configuration_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -17440,61 +17440,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -17512,7 +17512,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -17520,15 +17520,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.NamespaceDescriptor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NamespaceDescriptor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NamespaceDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NamespaceDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -17541,19 +17541,19 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getConfigurationFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        name_ = com.google.protobuf.ByteString.EMPTY;
+        name_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (configurationBuilder_ == null) {
           configuration_ = java.util.Collections.emptyList();
@@ -17564,7 +17564,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_NamespaceDescriptor_descriptor;
       }
@@ -17607,29 +17607,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor)other);
         } else {
@@ -17662,7 +17662,7 @@ public final class HBaseProtos {
               configuration_ = other.configuration_;
               bitField0_ = (bitField0_ & ~0x00000002);
               configurationBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getConfigurationFieldBuilder() : null;
             } else {
               configurationBuilder_.addAllMessages(other.configuration_);
@@ -17687,13 +17687,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -17705,7 +17705,7 @@ public final class HBaseProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString name_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString name_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes name = 1;
        */
@@ -17715,13 +17715,13 @@ public final class HBaseProtos {
       /**
        * required bytes name = 1;
        */
-      public com.google.protobuf.ByteString getName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getName() {
         return name_;
       }
       /**
        * required bytes name = 1;
        */
-      public Builder setName(com.google.protobuf.ByteString value) {
+      public Builder setName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -17749,7 +17749,7 @@ public final class HBaseProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> configurationBuilder_;
 
       /**
@@ -17881,7 +17881,7 @@ public final class HBaseProtos {
           java.lang.Iterable values) {
         if (configurationBuilder_ == null) {
           ensureConfigurationIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, configuration_);
           onChanged();
         } else {
@@ -17965,11 +17965,11 @@ public final class HBaseProtos {
            getConfigurationBuilderList() {
         return getConfigurationFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
           getConfigurationFieldBuilder() {
         if (configurationBuilder_ == null) {
-          configurationBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          configurationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder>(
                   configuration_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -17980,12 +17980,12 @@ public final class HBaseProtos {
         return configurationBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -18003,22 +18003,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NamespaceDescriptor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NamespaceDescriptor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -18030,7 +18030,7 @@ public final class HBaseProtos {
 
   public interface VersionInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.VersionInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string version = 1;
@@ -18043,7 +18043,7 @@ public final class HBaseProtos {
     /**
      * required string version = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getVersionBytes();
 
     /**
@@ -18057,7 +18057,7 @@ public final class HBaseProtos {
     /**
      * required string url = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUrlBytes();
 
     /**
@@ -18071,7 +18071,7 @@ public final class HBaseProtos {
     /**
      * required string revision = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRevisionBytes();
 
     /**
@@ -18085,7 +18085,7 @@ public final class HBaseProtos {
     /**
      * required string user = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUserBytes();
 
     /**
@@ -18099,7 +18099,7 @@ public final class HBaseProtos {
     /**
      * required string date = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getDateBytes();
 
     /**
@@ -18113,7 +18113,7 @@ public final class HBaseProtos {
     /**
      * required string src_checksum = 6;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSrcChecksumBytes();
 
     /**
@@ -18142,11 +18142,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.VersionInfo}
    */
   public  static final class VersionInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.VersionInfo)
       VersionInfoOrBuilder {
     // Use VersionInfo.newBuilder() to construct.
-    private VersionInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private VersionInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private VersionInfo() {
@@ -18161,18 +18161,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private VersionInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -18189,37 +18189,37 @@ public final class HBaseProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               version_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               url_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               revision_ = bs;
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               user_ = bs;
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000010;
               date_ = bs;
               break;
             }
             case 50: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000020;
               srcChecksum_ = bs;
               break;
@@ -18236,22 +18236,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_VersionInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_VersionInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -18275,8 +18275,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           version_ = s;
@@ -18287,17 +18287,17 @@ public final class HBaseProtos {
     /**
      * required string version = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getVersionBytes() {
       java.lang.Object ref = version_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         version_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18317,8 +18317,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           url_ = s;
@@ -18329,17 +18329,17 @@ public final class HBaseProtos {
     /**
      * required string url = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUrlBytes() {
       java.lang.Object ref = url_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         url_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18359,8 +18359,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           revision_ = s;
@@ -18371,17 +18371,17 @@ public final class HBaseProtos {
     /**
      * required string revision = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRevisionBytes() {
       java.lang.Object ref = revision_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         revision_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18401,8 +18401,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           user_ = s;
@@ -18413,17 +18413,17 @@ public final class HBaseProtos {
     /**
      * required string user = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUserBytes() {
       java.lang.Object ref = user_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         user_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18443,8 +18443,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           date_ = s;
@@ -18455,17 +18455,17 @@ public final class HBaseProtos {
     /**
      * required string date = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getDateBytes() {
       java.lang.Object ref = date_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         date_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18485,8 +18485,8 @@ public final class HBaseProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           srcChecksum_ = s;
@@ -18497,17 +18497,17 @@ public final class HBaseProtos {
     /**
      * required string src_checksum = 6;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getSrcChecksumBytes() {
       java.lang.Object ref = srcChecksum_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         srcChecksum_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -18575,25 +18575,25 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, version_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, version_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, url_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, url_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, revision_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, revision_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, user_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, user_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, date_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, date_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 6, srcChecksum_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 6, srcChecksum_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeUInt32(7, versionMajor_);
@@ -18610,29 +18610,29 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, version_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, version_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, url_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, url_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, revision_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, revision_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, user_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, user_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, date_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, date_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, srcChecksum_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(6, srcChecksum_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(7, versionMajor_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(8, versionMinor_);
       }
       size += unknownFields.getSerializedSize();
@@ -18741,61 +18741,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -18813,7 +18813,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -18825,15 +18825,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.VersionInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.VersionInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_VersionInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_VersionInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -18846,12 +18846,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -18876,7 +18876,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_VersionInfo_descriptor;
       }
@@ -18938,29 +18938,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo)other);
         } else {
@@ -19035,13 +19035,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -19066,8 +19066,8 @@ public final class HBaseProtos {
       public java.lang.String getVersion() {
         java.lang.Object ref = version_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             version_ = s;
@@ -19080,17 +19080,17 @@ public final class HBaseProtos {
       /**
        * required string version = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getVersionBytes() {
         java.lang.Object ref = version_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           version_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19119,7 +19119,7 @@ public final class HBaseProtos {
        * required string version = 1;
        */
       public Builder setVersionBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19142,8 +19142,8 @@ public final class HBaseProtos {
       public java.lang.String getUrl() {
         java.lang.Object ref = url_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             url_ = s;
@@ -19156,17 +19156,17 @@ public final class HBaseProtos {
       /**
        * required string url = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getUrlBytes() {
         java.lang.Object ref = url_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           url_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19195,7 +19195,7 @@ public final class HBaseProtos {
        * required string url = 2;
        */
       public Builder setUrlBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19218,8 +19218,8 @@ public final class HBaseProtos {
       public java.lang.String getRevision() {
         java.lang.Object ref = revision_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             revision_ = s;
@@ -19232,17 +19232,17 @@ public final class HBaseProtos {
       /**
        * required string revision = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getRevisionBytes() {
         java.lang.Object ref = revision_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           revision_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19271,7 +19271,7 @@ public final class HBaseProtos {
        * required string revision = 3;
        */
       public Builder setRevisionBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19294,8 +19294,8 @@ public final class HBaseProtos {
       public java.lang.String getUser() {
         java.lang.Object ref = user_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             user_ = s;
@@ -19308,17 +19308,17 @@ public final class HBaseProtos {
       /**
        * required string user = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getUserBytes() {
         java.lang.Object ref = user_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           user_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19347,7 +19347,7 @@ public final class HBaseProtos {
        * required string user = 4;
        */
       public Builder setUserBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19370,8 +19370,8 @@ public final class HBaseProtos {
       public java.lang.String getDate() {
         java.lang.Object ref = date_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             date_ = s;
@@ -19384,17 +19384,17 @@ public final class HBaseProtos {
       /**
        * required string date = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getDateBytes() {
         java.lang.Object ref = date_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           date_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19423,7 +19423,7 @@ public final class HBaseProtos {
        * required string date = 5;
        */
       public Builder setDateBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19446,8 +19446,8 @@ public final class HBaseProtos {
       public java.lang.String getSrcChecksum() {
         java.lang.Object ref = srcChecksum_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             srcChecksum_ = s;
@@ -19460,17 +19460,17 @@ public final class HBaseProtos {
       /**
        * required string src_checksum = 6;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getSrcChecksumBytes() {
         java.lang.Object ref = srcChecksum_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           srcChecksum_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19499,7 +19499,7 @@ public final class HBaseProtos {
        * required string src_checksum = 6;
        */
       public Builder setSrcChecksumBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19573,12 +19573,12 @@ public final class HBaseProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -19596,22 +19596,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public VersionInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new VersionInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -19623,7 +19623,7 @@ public final class HBaseProtos {
 
   public interface RegionServerInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionServerInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional int32 infoPort = 1;
@@ -19656,11 +19656,11 @@ public final class HBaseProtos {
    * Protobuf type {@code hbase.pb.RegionServerInfo}
    */
   public  static final class RegionServerInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionServerInfo)
       RegionServerInfoOrBuilder {
     // Use RegionServerInfo.newBuilder() to construct.
-    private RegionServerInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionServerInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionServerInfo() {
@@ -19668,18 +19668,18 @@ public final class HBaseProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionServerInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19715,22 +19715,22 @@ public final class HBaseProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionServerInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19790,7 +19790,7 @@ public final class HBaseProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(1, infoPort_);
@@ -19807,11 +19807,11 @@ public final class HBaseProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(1, infoPort_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getVersionInfo());
       }
       size += unknownFields.getSerializedSize();
@@ -19866,61 +19866,61 @@ public final class HBaseProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -19938,7 +19938,7 @@ public final class HBaseProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -19951,15 +19951,15 @@ public final class HBaseProtos {
      * Protobuf type {@code hbase.pb.RegionServerInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionServerInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionServerInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -19972,12 +19972,12 @@ public final class HBaseProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getVersionInfoFieldBuilder();
         }
@@ -19995,7 +19995,7 @@ public final class HBaseProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_RegionServerInfo_descriptor;
       }
@@ -20037,29 +20037,29 @@ public final class HBaseProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo)other);
         } else {
@@ -20091,13 +20091,13 @@ public final class HBaseProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20142,7 +20142,7 @@ public final class HBaseProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo versionInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder> versionInfoBuilder_;
       /**
        * optional .hbase.pb.VersionInfo version_info = 2;
@@ -20245,11 +20245,11 @@ public final class HBaseProtos {
       /**
        * optional .hbase.pb.VersionInfo version_info = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder> 
           getVersionInfoFieldBuilder() {
         if (versionInfoBuilder_ == null) {
-          versionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          versionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder>(
                   getVersionInfo(),
                   getParentForChildren(),
@@ -20259,12 +20259,12 @@ public final class HBaseProtos {
         return versionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -20282,22 +20282,22 @@ public final class HBaseProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionServerInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionServerInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -20307,137 +20307,137 @@ public final class HBaseProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableName_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TableName_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableSchema_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TableSchema_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TableState_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ColumnFamilySchema_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ColumnFamilySchema_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FavoredNodes_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FavoredNodes_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionSpecifier_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpecifier_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimeRange_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TimeRange_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ServerName_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ServerName_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Coprocessor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Coprocessor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NameStringPair_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NameStringPair_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NameBytesPair_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NameBytesPair_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BytesBytesPair_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BytesBytesPair_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NameInt64Pair_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NameInt64Pair_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotDescription_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureDescription_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ProcedureDescription_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_EmptyMsg_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_EmptyMsg_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_LongMsg_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_LongMsg_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DoubleMsg_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DoubleMsg_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BigDecimalMsg_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BigDecimalMsg_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UUID_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UUID_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NamespaceDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NamespaceDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_VersionInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_VersionInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionServerInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -20505,166 +20505,166 @@ public final class HBaseProtos {
       "e.shaded.protobuf.generatedB\013HBaseProtos" +
       "H\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_TableName_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_TableName_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TableName_descriptor,
         new java.lang.String[] { "Namespace", "Qualifier", });
     internal_static_hbase_pb_TableSchema_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_TableSchema_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TableSchema_descriptor,
         new java.lang.String[] { "TableName", "Attributes", "ColumnFamilies", "Configuration", });
     internal_static_hbase_pb_TableState_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_TableState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TableState_descriptor,
         new java.lang.String[] { "State", });
     internal_static_hbase_pb_ColumnFamilySchema_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_ColumnFamilySchema_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ColumnFamilySchema_descriptor,
         new java.lang.String[] { "Name", "Attributes", "Configuration", });
     internal_static_hbase_pb_RegionInfo_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_RegionInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionInfo_descriptor,
         new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", "ReplicaId", });
     internal_static_hbase_pb_FavoredNodes_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_FavoredNodes_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FavoredNodes_descriptor,
         new java.lang.String[] { "FavoredNode", });
     internal_static_hbase_pb_RegionSpecifier_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_RegionSpecifier_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionSpecifier_descriptor,
         new java.lang.String[] { "Type", "Value", });
     internal_static_hbase_pb_TimeRange_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_TimeRange_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TimeRange_descriptor,
         new java.lang.String[] { "From", "To", });
     internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_ColumnFamilyTimeRange_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ColumnFamilyTimeRange_descriptor,
         new java.lang.String[] { "ColumnFamily", "TimeRange", });
     internal_static_hbase_pb_ServerName_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_ServerName_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ServerName_descriptor,
         new java.lang.String[] { "HostName", "Port", "StartCode", });
     internal_static_hbase_pb_Coprocessor_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_Coprocessor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Coprocessor_descriptor,
         new java.lang.String[] { "Name", });
     internal_static_hbase_pb_NameStringPair_descriptor =
       getDescriptor().getMessageTypes().get(11);
     internal_static_hbase_pb_NameStringPair_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NameStringPair_descriptor,
         new java.lang.String[] { "Name", "Value", });
     internal_static_hbase_pb_NameBytesPair_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_NameBytesPair_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NameBytesPair_descriptor,
         new java.lang.String[] { "Name", "Value", });
     internal_static_hbase_pb_BytesBytesPair_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_hbase_pb_BytesBytesPair_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BytesBytesPair_descriptor,
         new java.lang.String[] { "First", "Second", });
     internal_static_hbase_pb_NameInt64Pair_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_NameInt64Pair_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NameInt64Pair_descriptor,
         new java.lang.String[] { "Name", "Value", });
     internal_static_hbase_pb_SnapshotDescription_descriptor =
       getDescriptor().getMessageTypes().get(15);
     internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotDescription_descriptor,
         new java.lang.String[] { "Name", "Table", "CreationTime", "Type", "Version", "Owner", });
     internal_static_hbase_pb_ProcedureDescription_descriptor =
       getDescriptor().getMessageTypes().get(16);
     internal_static_hbase_pb_ProcedureDescription_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ProcedureDescription_descriptor,
         new java.lang.String[] { "Signature", "Instance", "CreationTime", "Configuration", });
     internal_static_hbase_pb_EmptyMsg_descriptor =
       getDescriptor().getMessageTypes().get(17);
     internal_static_hbase_pb_EmptyMsg_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_EmptyMsg_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_LongMsg_descriptor =
       getDescriptor().getMessageTypes().get(18);
     internal_static_hbase_pb_LongMsg_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_LongMsg_descriptor,
         new java.lang.String[] { "LongMsg", });
     internal_static_hbase_pb_DoubleMsg_descriptor =
       getDescriptor().getMessageTypes().get(19);
     internal_static_hbase_pb_DoubleMsg_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DoubleMsg_descriptor,
         new java.lang.String[] { "DoubleMsg", });
     internal_static_hbase_pb_BigDecimalMsg_descriptor =
       getDescriptor().getMessageTypes().get(20);
     internal_static_hbase_pb_BigDecimalMsg_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BigDecimalMsg_descriptor,
         new java.lang.String[] { "BigdecimalMsg", });
     internal_static_hbase_pb_UUID_descriptor =
       getDescriptor().getMessageTypes().get(21);
     internal_static_hbase_pb_UUID_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UUID_descriptor,
         new java.lang.String[] { "LeastSigBits", "MostSigBits", });
     internal_static_hbase_pb_NamespaceDescriptor_descriptor =
       getDescriptor().getMessageTypes().get(22);
     internal_static_hbase_pb_NamespaceDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NamespaceDescriptor_descriptor,
         new java.lang.String[] { "Name", "Configuration", });
     internal_static_hbase_pb_VersionInfo_descriptor =
       getDescriptor().getMessageTypes().get(23);
     internal_static_hbase_pb_VersionInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_VersionInfo_descriptor,
         new java.lang.String[] { "Version", "Url", "Revision", "User", "Date", "SrcChecksum", "VersionMajor", "VersionMinor", });
     internal_static_hbase_pb_RegionServerInfo_descriptor =
       getDescriptor().getMessageTypes().get(24);
     internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionServerInfo_descriptor,
         new java.lang.String[] { "InfoPort", "VersionInfo", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
index 2a3d649..00e7080 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class HFileProtos {
   private HFileProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface FileInfoProtoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FileInfoProto)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.BytesBytesPair map_entry = 1;
@@ -50,11 +50,11 @@ public final class HFileProtos {
    * Protobuf type {@code hbase.pb.FileInfoProto}
    */
   public  static final class FileInfoProto extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FileInfoProto)
       FileInfoProtoOrBuilder {
     // Use FileInfoProto.newBuilder() to construct.
-    private FileInfoProto(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FileInfoProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FileInfoProto() {
@@ -62,18 +62,18 @@ public final class HFileProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FileInfoProto(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -100,10 +100,10 @@ public final class HFileProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -113,12 +113,12 @@ public final class HFileProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileInfoProto_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileInfoProto_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -176,7 +176,7 @@ public final class HFileProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < mapEntry_.size(); i++) {
         output.writeMessage(1, mapEntry_.get(i));
@@ -190,7 +190,7 @@ public final class HFileProtos {
 
       size = 0;
       for (int i = 0; i < mapEntry_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, mapEntry_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -233,61 +233,61 @@ public final class HFileProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -305,7 +305,7 @@ public final class HFileProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -317,15 +317,15 @@ public final class HFileProtos {
      * Protobuf type {@code hbase.pb.FileInfoProto}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FileInfoProto)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProtoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileInfoProto_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileInfoProto_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -338,12 +338,12 @@ public final class HFileProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getMapEntryFieldBuilder();
         }
@@ -359,7 +359,7 @@ public final class HFileProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileInfoProto_descriptor;
       }
@@ -396,29 +396,29 @@ public final class HFileProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto)other);
         } else {
@@ -448,7 +448,7 @@ public final class HFileProtos {
               mapEntry_ = other.mapEntry_;
               bitField0_ = (bitField0_ & ~0x00000001);
               mapEntryBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getMapEntryFieldBuilder() : null;
             } else {
               mapEntryBuilder_.addAllMessages(other.mapEntry_);
@@ -470,13 +470,13 @@ public final class HFileProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileInfoProto) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -497,7 +497,7 @@ public final class HFileProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> mapEntryBuilder_;
 
       /**
@@ -629,7 +629,7 @@ public final class HFileProtos {
           java.lang.Iterable values) {
         if (mapEntryBuilder_ == null) {
           ensureMapEntryIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, mapEntry_);
           onChanged();
         } else {
@@ -713,11 +713,11 @@ public final class HFileProtos {
            getMapEntryBuilderList() {
         return getMapEntryFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
           getMapEntryFieldBuilder() {
         if (mapEntryBuilder_ == null) {
-          mapEntryBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          mapEntryBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>(
                   mapEntry_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -728,12 +728,12 @@ public final class HFileProtos {
         return mapEntryBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -751,22 +751,22 @@ public final class HFileProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FileInfoProto parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FileInfoProto(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -778,7 +778,7 @@ public final class HFileProtos {
 
   public interface FileTrailerProtoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FileTrailerProto)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 file_info_offset = 1;
@@ -881,7 +881,7 @@ public final class HFileProtos {
     /**
      * optional string comparator_class_name = 11;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getComparatorClassNameBytes();
 
     /**
@@ -900,7 +900,7 @@ public final class HFileProtos {
     /**
      * optional bytes encryption_key = 13;
      */
-    com.google.protobuf.ByteString getEncryptionKey();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncryptionKey();
   }
   /**
    * 
@@ -910,11 +910,11 @@ public final class HFileProtos {
    * Protobuf type {@code hbase.pb.FileTrailerProto}
    */
   public  static final class FileTrailerProto extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FileTrailerProto)
       FileTrailerProtoOrBuilder {
     // Use FileTrailerProto.newBuilder() to construct.
-    private FileTrailerProto(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FileTrailerProto(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FileTrailerProto() {
@@ -930,22 +930,22 @@ public final class HFileProtos {
       lastDataBlockOffset_ = 0L;
       comparatorClassName_ = "";
       compressionCodec_ = 0;
-      encryptionKey_ = com.google.protobuf.ByteString.EMPTY;
+      encryptionKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FileTrailerProto(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1012,7 +1012,7 @@ public final class HFileProtos {
               break;
             }
             case 90: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000400;
               comparatorClassName_ = bs;
               break;
@@ -1029,22 +1029,22 @@ public final class HFileProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileTrailerProto_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileTrailerProto_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1218,8 +1218,8 @@ public final class HFileProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           comparatorClassName_ = s;
@@ -1230,17 +1230,17 @@ public final class HFileProtos {
     /**
      * optional string comparator_class_name = 11;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getComparatorClassNameBytes() {
       java.lang.Object ref = comparatorClassName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         comparatorClassName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -1260,7 +1260,7 @@ public final class HFileProtos {
     }
 
     public static final int ENCRYPTION_KEY_FIELD_NUMBER = 13;
-    private com.google.protobuf.ByteString encryptionKey_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encryptionKey_;
     /**
      * optional bytes encryption_key = 13;
      */
@@ -1270,7 +1270,7 @@ public final class HFileProtos {
     /**
      * optional bytes encryption_key = 13;
      */
-    public com.google.protobuf.ByteString getEncryptionKey() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncryptionKey() {
       return encryptionKey_;
     }
 
@@ -1284,7 +1284,7 @@ public final class HFileProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, fileInfoOffset_);
@@ -1317,7 +1317,7 @@ public final class HFileProtos {
         output.writeUInt64(10, lastDataBlockOffset_);
       }
       if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 11, comparatorClassName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 11, comparatorClassName_);
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
         output.writeUInt32(12, compressionCodec_);
@@ -1334,54 +1334,54 @@ public final class HFileProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, fileInfoOffset_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, loadOnOpenDataOffset_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, uncompressedDataIndexSize_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, totalUncompressedBytes_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(5, dataIndexCount_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(6, metaIndexCount_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(7, entryCount_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(8, numDataIndexLevels_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(9, firstDataBlockOffset_);
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(10, lastDataBlockOffset_);
       }
       if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(11, comparatorClassName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(11, comparatorClassName_);
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(12, compressionCodec_);
       }
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(13, encryptionKey_);
       }
       size += unknownFields.getSerializedSize();
@@ -1479,22 +1479,22 @@ public final class HFileProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasFileInfoOffset()) {
         hash = (37 * hash) + FILE_INFO_OFFSET_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getFileInfoOffset());
       }
       if (hasLoadOnOpenDataOffset()) {
         hash = (37 * hash) + LOAD_ON_OPEN_DATA_OFFSET_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLoadOnOpenDataOffset());
       }
       if (hasUncompressedDataIndexSize()) {
         hash = (37 * hash) + UNCOMPRESSED_DATA_INDEX_SIZE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getUncompressedDataIndexSize());
       }
       if (hasTotalUncompressedBytes()) {
         hash = (37 * hash) + TOTAL_UNCOMPRESSED_BYTES_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTotalUncompressedBytes());
       }
       if (hasDataIndexCount()) {
@@ -1507,7 +1507,7 @@ public final class HFileProtos {
       }
       if (hasEntryCount()) {
         hash = (37 * hash) + ENTRY_COUNT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getEntryCount());
       }
       if (hasNumDataIndexLevels()) {
@@ -1516,12 +1516,12 @@ public final class HFileProtos {
       }
       if (hasFirstDataBlockOffset()) {
         hash = (37 * hash) + FIRST_DATA_BLOCK_OFFSET_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getFirstDataBlockOffset());
       }
       if (hasLastDataBlockOffset()) {
         hash = (37 * hash) + LAST_DATA_BLOCK_OFFSET_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastDataBlockOffset());
       }
       if (hasComparatorClassName()) {
@@ -1542,61 +1542,61 @@ public final class HFileProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1614,7 +1614,7 @@ public final class HFileProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1626,15 +1626,15 @@ public final class HFileProtos {
      * Protobuf type {@code hbase.pb.FileTrailerProto}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FileTrailerProto)
         org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProtoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileTrailerProto_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileTrailerProto_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1647,12 +1647,12 @@ public final class HFileProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -1682,12 +1682,12 @@ public final class HFileProtos {
         bitField0_ = (bitField0_ & ~0x00000400);
         compressionCodec_ = 0;
         bitField0_ = (bitField0_ & ~0x00000800);
-        encryptionKey_ = com.google.protobuf.ByteString.EMPTY;
+        encryptionKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00001000);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.internal_static_hbase_pb_FileTrailerProto_descriptor;
       }
@@ -1769,29 +1769,29 @@ public final class HFileProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto)other);
         } else {
@@ -1853,13 +1853,13 @@ public final class HFileProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos.FileTrailerProto) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2204,8 +2204,8 @@ public final class HFileProtos {
       public java.lang.String getComparatorClassName() {
         java.lang.Object ref = comparatorClassName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             comparatorClassName_ = s;
@@ -2218,17 +2218,17 @@ public final class HFileProtos {
       /**
        * optional string comparator_class_name = 11;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getComparatorClassNameBytes() {
         java.lang.Object ref = comparatorClassName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           comparatorClassName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -2257,7 +2257,7 @@ public final class HFileProtos {
        * optional string comparator_class_name = 11;
        */
       public Builder setComparatorClassNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2299,7 +2299,7 @@ public final class HFileProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString encryptionKey_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encryptionKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes encryption_key = 13;
        */
@@ -2309,13 +2309,13 @@ public final class HFileProtos {
       /**
        * optional bytes encryption_key = 13;
        */
-      public com.google.protobuf.ByteString getEncryptionKey() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncryptionKey() {
         return encryptionKey_;
       }
       /**
        * optional bytes encryption_key = 13;
        */
-      public Builder setEncryptionKey(com.google.protobuf.ByteString value) {
+      public Builder setEncryptionKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2334,12 +2334,12 @@ public final class HFileProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2357,22 +2357,22 @@ public final class HFileProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FileTrailerProto parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FileTrailerProto(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2382,22 +2382,22 @@ public final class HFileProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FileInfoProto_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FileInfoProto_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FileTrailerProto_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FileTrailerProto_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -2417,29 +2417,29 @@ public final class HFileProtos {
       "oop.hbase.shaded.protobuf.generatedB\013HFi" +
       "leProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_FileInfoProto_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_FileInfoProto_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FileInfoProto_descriptor,
         new java.lang.String[] { "MapEntry", });
     internal_static_hbase_pb_FileTrailerProto_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_FileTrailerProto_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FileTrailerProto_descriptor,
         new java.lang.String[] { "FileInfoOffset", "LoadOnOpenDataOffset", "UncompressedDataIndexSize", "TotalUncompressedBytes", "DataIndexCount", "MetaIndexCount", "EntryCount", "NumDataIndexLevels", "FirstDataBlockOffset", "LastDataBlockOffset", "ComparatorClassName", "CompressionCodec", "EncryptionKey", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
index 2226f76..0b941d8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class LoadBalancerProtos {
   private LoadBalancerProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface LoadBalancerStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.LoadBalancerState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool balancer_on = 1;
@@ -31,11 +31,11 @@ public final class LoadBalancerProtos {
    * Protobuf type {@code hbase.pb.LoadBalancerState}
    */
   public  static final class LoadBalancerState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.LoadBalancerState)
       LoadBalancerStateOrBuilder {
     // Use LoadBalancerState.newBuilder() to construct.
-    private LoadBalancerState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private LoadBalancerState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private LoadBalancerState() {
@@ -43,18 +43,18 @@ public final class LoadBalancerProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private LoadBalancerState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -77,22 +77,22 @@ public final class LoadBalancerProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.internal_static_hbase_pb_LoadBalancerState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.internal_static_hbase_pb_LoadBalancerState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -125,7 +125,7 @@ public final class LoadBalancerProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, balancerOn_);
@@ -139,7 +139,7 @@ public final class LoadBalancerProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, balancerOn_);
       }
       size += unknownFields.getSerializedSize();
@@ -177,7 +177,7 @@ public final class LoadBalancerProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasBalancerOn()) {
         hash = (37 * hash) + BALANCER_ON_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getBalancerOn());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -186,61 +186,61 @@ public final class LoadBalancerProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -258,7 +258,7 @@ public final class LoadBalancerProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -266,15 +266,15 @@ public final class LoadBalancerProtos {
      * Protobuf type {@code hbase.pb.LoadBalancerState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.LoadBalancerState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.internal_static_hbase_pb_LoadBalancerState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.internal_static_hbase_pb_LoadBalancerState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -287,12 +287,12 @@ public final class LoadBalancerProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -303,7 +303,7 @@ public final class LoadBalancerProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.internal_static_hbase_pb_LoadBalancerState_descriptor;
       }
@@ -337,29 +337,29 @@ public final class LoadBalancerProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState)other);
         } else {
@@ -383,13 +383,13 @@ public final class LoadBalancerProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos.LoadBalancerState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -433,12 +433,12 @@ public final class LoadBalancerProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -456,22 +456,22 @@ public final class LoadBalancerProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public LoadBalancerState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new LoadBalancerState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -481,17 +481,17 @@ public final class LoadBalancerProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_LoadBalancerState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_LoadBalancerState_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -500,22 +500,22 @@ public final class LoadBalancerProtos {
       "apache.hadoop.hbase.shaded.protobuf.gene" +
       "ratedB\022LoadBalancerProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_LoadBalancerState_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_LoadBalancerState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_LoadBalancerState_descriptor,
         new java.lang.String[] { "BalancerOn", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
index b5abf42..09605a2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class MapReduceProtos {
   private MapReduceProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface ScanMetricsOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ScanMetrics)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.NameInt64Pair metrics = 1;
@@ -46,11 +46,11 @@ public final class MapReduceProtos {
    * Protobuf type {@code hbase.pb.ScanMetrics}
    */
   public  static final class ScanMetrics extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ScanMetrics)
       ScanMetricsOrBuilder {
     // Use ScanMetrics.newBuilder() to construct.
-    private ScanMetrics(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ScanMetrics(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ScanMetrics() {
@@ -58,18 +58,18 @@ public final class MapReduceProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ScanMetrics(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -96,10 +96,10 @@ public final class MapReduceProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -109,12 +109,12 @@ public final class MapReduceProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_ScanMetrics_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_ScanMetrics_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -166,7 +166,7 @@ public final class MapReduceProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < metrics_.size(); i++) {
         output.writeMessage(1, metrics_.get(i));
@@ -180,7 +180,7 @@ public final class MapReduceProtos {
 
       size = 0;
       for (int i = 0; i < metrics_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, metrics_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -223,61 +223,61 @@ public final class MapReduceProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -295,7 +295,7 @@ public final class MapReduceProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -303,15 +303,15 @@ public final class MapReduceProtos {
      * Protobuf type {@code hbase.pb.ScanMetrics}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ScanMetrics)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_ScanMetrics_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_ScanMetrics_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -324,12 +324,12 @@ public final class MapReduceProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getMetricsFieldBuilder();
         }
@@ -345,7 +345,7 @@ public final class MapReduceProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_ScanMetrics_descriptor;
       }
@@ -382,29 +382,29 @@ public final class MapReduceProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics)other);
         } else {
@@ -434,7 +434,7 @@ public final class MapReduceProtos {
               metrics_ = other.metrics_;
               bitField0_ = (bitField0_ & ~0x00000001);
               metricsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getMetricsFieldBuilder() : null;
             } else {
               metricsBuilder_.addAllMessages(other.metrics_);
@@ -451,13 +451,13 @@ public final class MapReduceProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -478,7 +478,7 @@ public final class MapReduceProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64PairOrBuilder> metricsBuilder_;
 
       /**
@@ -610,7 +610,7 @@ public final class MapReduceProtos {
           java.lang.Iterable values) {
         if (metricsBuilder_ == null) {
           ensureMetricsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, metrics_);
           onChanged();
         } else {
@@ -694,11 +694,11 @@ public final class MapReduceProtos {
            getMetricsBuilderList() {
         return getMetricsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64PairOrBuilder> 
           getMetricsFieldBuilder() {
         if (metricsBuilder_ == null) {
-          metricsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          metricsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64PairOrBuilder>(
                   metrics_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -709,12 +709,12 @@ public final class MapReduceProtos {
         return metricsBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -732,22 +732,22 @@ public final class MapReduceProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ScanMetrics parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ScanMetrics(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -759,7 +759,7 @@ public final class MapReduceProtos {
 
   public interface TableSnapshotRegionSplitOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TableSnapshotRegionSplit)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated string locations = 2;
@@ -777,7 +777,7 @@ public final class MapReduceProtos {
     /**
      * repeated string locations = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getLocationsBytes(int index);
 
     /**
@@ -810,30 +810,30 @@ public final class MapReduceProtos {
    * Protobuf type {@code hbase.pb.TableSnapshotRegionSplit}
    */
   public  static final class TableSnapshotRegionSplit extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TableSnapshotRegionSplit)
       TableSnapshotRegionSplitOrBuilder {
     // Use TableSnapshotRegionSplit.newBuilder() to construct.
-    private TableSnapshotRegionSplit(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TableSnapshotRegionSplit(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TableSnapshotRegionSplit() {
-      locations_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      locations_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TableSnapshotRegionSplit(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -850,9 +850,9 @@ public final class MapReduceProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                locations_ = new com.google.protobuf.LazyStringArrayList();
+                locations_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                 mutable_bitField0_ |= 0x00000001;
               }
               locations_.add(bs);
@@ -886,10 +886,10 @@ public final class MapReduceProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -899,12 +899,12 @@ public final class MapReduceProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_TableSnapshotRegionSplit_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_TableSnapshotRegionSplit_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -913,11 +913,11 @@ public final class MapReduceProtos {
 
     private int bitField0_;
     public static final int LOCATIONS_FIELD_NUMBER = 2;
-    private com.google.protobuf.LazyStringList locations_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList locations_;
     /**
      * repeated string locations = 2;
      */
-    public com.google.protobuf.ProtocolStringList
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
         getLocationsList() {
       return locations_;
     }
@@ -936,7 +936,7 @@ public final class MapReduceProtos {
     /**
      * repeated string locations = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getLocationsBytes(int index) {
       return locations_.getByteString(index);
     }
@@ -1005,10 +1005,10 @@ public final class MapReduceProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < locations_.size(); i++) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, locations_.getRaw(i));
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, locations_.getRaw(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(3, getTable());
@@ -1033,11 +1033,11 @@ public final class MapReduceProtos {
         size += 1 * getLocationsList().size();
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getTable());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getRegion());
       }
       size += unknownFields.getSerializedSize();
@@ -1098,61 +1098,61 @@ public final class MapReduceProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1170,7 +1170,7 @@ public final class MapReduceProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1178,15 +1178,15 @@ public final class MapReduceProtos {
      * Protobuf type {@code hbase.pb.TableSnapshotRegionSplit}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TableSnapshotRegionSplit)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplitOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_TableSnapshotRegionSplit_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_TableSnapshotRegionSplit_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1199,12 +1199,12 @@ public final class MapReduceProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableFieldBuilder();
           getRegionFieldBuilder();
@@ -1212,7 +1212,7 @@ public final class MapReduceProtos {
       }
       public Builder clear() {
         super.clear();
-        locations_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        locations_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (tableBuilder_ == null) {
           table_ = null;
@@ -1229,7 +1229,7 @@ public final class MapReduceProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.internal_static_hbase_pb_TableSnapshotRegionSplit_descriptor;
       }
@@ -1280,29 +1280,29 @@ public final class MapReduceProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit)other);
         } else {
@@ -1349,13 +1349,13 @@ public final class MapReduceProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1367,17 +1367,17 @@ public final class MapReduceProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.LazyStringList locations_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList locations_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureLocationsIsMutable() {
         if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          locations_ = new com.google.protobuf.LazyStringArrayList(locations_);
+          locations_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(locations_);
           bitField0_ |= 0x00000001;
          }
       }
       /**
        * repeated string locations = 2;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getLocationsList() {
         return locations_.getUnmodifiableView();
       }
@@ -1396,7 +1396,7 @@ public final class MapReduceProtos {
       /**
        * repeated string locations = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getLocationsBytes(int index) {
         return locations_.getByteString(index);
       }
@@ -1432,7 +1432,7 @@ public final class MapReduceProtos {
       public Builder addAllLocations(
           java.lang.Iterable values) {
         ensureLocationsIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, locations_);
         onChanged();
         return this;
@@ -1441,7 +1441,7 @@ public final class MapReduceProtos {
        * repeated string locations = 2;
        */
       public Builder clearLocations() {
-        locations_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        locations_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         onChanged();
         return this;
@@ -1450,7 +1450,7 @@ public final class MapReduceProtos {
        * repeated string locations = 2;
        */
       public Builder addLocationsBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1461,7 +1461,7 @@ public final class MapReduceProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema table_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableBuilder_;
       /**
        * optional .hbase.pb.TableSchema table = 3;
@@ -1564,11 +1564,11 @@ public final class MapReduceProtos {
       /**
        * optional .hbase.pb.TableSchema table = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableFieldBuilder() {
         if (tableBuilder_ == null) {
-          tableBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTable(),
                   getParentForChildren(),
@@ -1579,7 +1579,7 @@ public final class MapReduceProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionBuilder_;
       /**
        * optional .hbase.pb.RegionInfo region = 4;
@@ -1682,11 +1682,11 @@ public final class MapReduceProtos {
       /**
        * optional .hbase.pb.RegionInfo region = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -1696,12 +1696,12 @@ public final class MapReduceProtos {
         return regionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1719,22 +1719,22 @@ public final class MapReduceProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TableSnapshotRegionSplit parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TableSnapshotRegionSplit(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1744,22 +1744,22 @@ public final class MapReduceProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ScanMetrics_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ScanMetrics_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableSnapshotRegionSplit_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TableSnapshotRegionSplit_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -1772,29 +1772,29 @@ public final class MapReduceProtos {
       "p.hbase.shaded.protobuf.generatedB\017MapRe" +
       "duceProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_ScanMetrics_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_ScanMetrics_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ScanMetrics_descriptor,
         new java.lang.String[] { "Metrics", });
     internal_static_hbase_pb_TableSnapshotRegionSplit_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_TableSnapshotRegionSplit_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TableSnapshotRegionSplit_descriptor,
         new java.lang.String[] { "Locations", "Table", "Region", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index bc8d40b..9942cd2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -6,19 +6,19 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class MasterProcedureProtos {
   private MasterProcedureProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * Protobuf enum {@code hbase.pb.CreateTableState}
    */
   public enum CreateTableState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * CREATE_TABLE_PRE_OPERATION = 1;
      */
@@ -95,27 +95,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         CreateTableState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public CreateTableState findValueByNumber(int number) {
               return CreateTableState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -123,7 +123,7 @@ public final class MasterProcedureProtos {
     private static final CreateTableState[] VALUES = values();
 
     public static CreateTableState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -144,7 +144,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.ModifyTableState}
    */
   public enum ModifyTableState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * MODIFY_TABLE_PREPARE = 1;
      */
@@ -230,27 +230,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ModifyTableState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ModifyTableState findValueByNumber(int number) {
               return ModifyTableState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(1);
     }
@@ -258,7 +258,7 @@ public final class MasterProcedureProtos {
     private static final ModifyTableState[] VALUES = values();
 
     public static ModifyTableState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -279,7 +279,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.TruncateTableState}
    */
   public enum TruncateTableState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * TRUNCATE_TABLE_PRE_OPERATION = 1;
      */
@@ -365,27 +365,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         TruncateTableState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public TruncateTableState findValueByNumber(int number) {
               return TruncateTableState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(2);
     }
@@ -393,7 +393,7 @@ public final class MasterProcedureProtos {
     private static final TruncateTableState[] VALUES = values();
 
     public static TruncateTableState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -414,7 +414,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.DeleteTableState}
    */
   public enum DeleteTableState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * DELETE_TABLE_PRE_OPERATION = 1;
      */
@@ -491,27 +491,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         DeleteTableState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public DeleteTableState findValueByNumber(int number) {
               return DeleteTableState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(3);
     }
@@ -519,7 +519,7 @@ public final class MasterProcedureProtos {
     private static final DeleteTableState[] VALUES = values();
 
     public static DeleteTableState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -540,7 +540,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.CreateNamespaceState}
    */
   public enum CreateNamespaceState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * CREATE_NAMESPACE_PREPARE = 1;
      */
@@ -608,27 +608,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         CreateNamespaceState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public CreateNamespaceState findValueByNumber(int number) {
               return CreateNamespaceState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(4);
     }
@@ -636,7 +636,7 @@ public final class MasterProcedureProtos {
     private static final CreateNamespaceState[] VALUES = values();
 
     public static CreateNamespaceState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -657,7 +657,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.ModifyNamespaceState}
    */
   public enum ModifyNamespaceState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * MODIFY_NAMESPACE_PREPARE = 1;
      */
@@ -707,27 +707,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ModifyNamespaceState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ModifyNamespaceState findValueByNumber(int number) {
               return ModifyNamespaceState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(5);
     }
@@ -735,7 +735,7 @@ public final class MasterProcedureProtos {
     private static final ModifyNamespaceState[] VALUES = values();
 
     public static ModifyNamespaceState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -756,7 +756,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.DeleteNamespaceState}
    */
   public enum DeleteNamespaceState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * DELETE_NAMESPACE_PREPARE = 1;
      */
@@ -824,27 +824,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         DeleteNamespaceState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public DeleteNamespaceState findValueByNumber(int number) {
               return DeleteNamespaceState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(6);
     }
@@ -852,7 +852,7 @@ public final class MasterProcedureProtos {
     private static final DeleteNamespaceState[] VALUES = values();
 
     public static DeleteNamespaceState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -873,7 +873,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.AddColumnFamilyState}
    */
   public enum AddColumnFamilyState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * ADD_COLUMN_FAMILY_PREPARE = 1;
      */
@@ -941,27 +941,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         AddColumnFamilyState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public AddColumnFamilyState findValueByNumber(int number) {
               return AddColumnFamilyState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(7);
     }
@@ -969,7 +969,7 @@ public final class MasterProcedureProtos {
     private static final AddColumnFamilyState[] VALUES = values();
 
     public static AddColumnFamilyState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -990,7 +990,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.ModifyColumnFamilyState}
    */
   public enum ModifyColumnFamilyState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * MODIFY_COLUMN_FAMILY_PREPARE = 1;
      */
@@ -1058,27 +1058,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ModifyColumnFamilyState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ModifyColumnFamilyState findValueByNumber(int number) {
               return ModifyColumnFamilyState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(8);
     }
@@ -1086,7 +1086,7 @@ public final class MasterProcedureProtos {
     private static final ModifyColumnFamilyState[] VALUES = values();
 
     public static ModifyColumnFamilyState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1107,7 +1107,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.DeleteColumnFamilyState}
    */
   public enum DeleteColumnFamilyState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * DELETE_COLUMN_FAMILY_PREPARE = 1;
      */
@@ -1184,27 +1184,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         DeleteColumnFamilyState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public DeleteColumnFamilyState findValueByNumber(int number) {
               return DeleteColumnFamilyState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(9);
     }
@@ -1212,7 +1212,7 @@ public final class MasterProcedureProtos {
     private static final DeleteColumnFamilyState[] VALUES = values();
 
     public static DeleteColumnFamilyState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1233,7 +1233,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.EnableTableState}
    */
   public enum EnableTableState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * ENABLE_TABLE_PREPARE = 1;
      */
@@ -1310,27 +1310,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         EnableTableState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public EnableTableState findValueByNumber(int number) {
               return EnableTableState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(10);
     }
@@ -1338,7 +1338,7 @@ public final class MasterProcedureProtos {
     private static final EnableTableState[] VALUES = values();
 
     public static EnableTableState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1359,7 +1359,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.DisableTableState}
    */
   public enum DisableTableState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * DISABLE_TABLE_PREPARE = 1;
      */
@@ -1436,27 +1436,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         DisableTableState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public DisableTableState findValueByNumber(int number) {
               return DisableTableState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(11);
     }
@@ -1464,7 +1464,7 @@ public final class MasterProcedureProtos {
     private static final DisableTableState[] VALUES = values();
 
     public static DisableTableState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1485,7 +1485,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.CloneSnapshotState}
    */
   public enum CloneSnapshotState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * CLONE_SNAPSHOT_PRE_OPERATION = 1;
      */
@@ -1562,27 +1562,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         CloneSnapshotState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public CloneSnapshotState findValueByNumber(int number) {
               return CloneSnapshotState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(12);
     }
@@ -1590,7 +1590,7 @@ public final class MasterProcedureProtos {
     private static final CloneSnapshotState[] VALUES = values();
 
     public static CloneSnapshotState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1611,7 +1611,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.RestoreSnapshotState}
    */
   public enum RestoreSnapshotState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * RESTORE_SNAPSHOT_PRE_OPERATION = 1;
      */
@@ -1670,27 +1670,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         RestoreSnapshotState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public RestoreSnapshotState findValueByNumber(int number) {
               return RestoreSnapshotState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(13);
     }
@@ -1698,7 +1698,7 @@ public final class MasterProcedureProtos {
     private static final RestoreSnapshotState[] VALUES = values();
 
     public static RestoreSnapshotState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1719,7 +1719,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.DispatchMergingRegionsState}
    */
   public enum DispatchMergingRegionsState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * DISPATCH_MERGING_REGIONS_PREPARE = 1;
      */
@@ -1787,27 +1787,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         DispatchMergingRegionsState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public DispatchMergingRegionsState findValueByNumber(int number) {
               return DispatchMergingRegionsState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(14);
     }
@@ -1815,7 +1815,7 @@ public final class MasterProcedureProtos {
     private static final DispatchMergingRegionsState[] VALUES = values();
 
     public static DispatchMergingRegionsState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1836,7 +1836,7 @@ public final class MasterProcedureProtos {
    * Protobuf enum {@code hbase.pb.ServerCrashState}
    */
   public enum ServerCrashState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * SERVER_CRASH_START = 1;
      */
@@ -1948,27 +1948,27 @@ public final class MasterProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ServerCrashState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ServerCrashState findValueByNumber(int number) {
               return ServerCrashState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(15);
     }
@@ -1976,7 +1976,7 @@ public final class MasterProcedureProtos {
     private static final ServerCrashState[] VALUES = values();
 
     public static ServerCrashState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -1995,7 +1995,7 @@ public final class MasterProcedureProtos {
 
   public interface CreateTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -2051,11 +2051,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.CreateTableStateData}
    */
   public  static final class CreateTableStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CreateTableStateData)
       CreateTableStateDataOrBuilder {
     // Use CreateTableStateData.newBuilder() to construct.
-    private CreateTableStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CreateTableStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CreateTableStateData() {
@@ -2063,18 +2063,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CreateTableStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2127,10 +2127,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -2140,12 +2140,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateTableStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2262,7 +2262,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -2282,15 +2282,15 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableSchema());
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, regionInfo_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -2351,61 +2351,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2423,7 +2423,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2431,15 +2431,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.CreateTableStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CreateTableStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateTableStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2452,12 +2452,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableSchemaFieldBuilder();
@@ -2487,7 +2487,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateTableStateData_descriptor;
       }
@@ -2542,29 +2542,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData)other);
         } else {
@@ -2600,7 +2600,7 @@ public final class MasterProcedureProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000004);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -2634,13 +2634,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2653,7 +2653,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -2756,11 +2756,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -2771,7 +2771,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema table_schema = 2;
@@ -2874,11 +2874,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableSchema table_schema = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTableSchema(),
                   getParentForChildren(),
@@ -2897,7 +2897,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -3029,7 +3029,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -3113,11 +3113,11 @@ public final class MasterProcedureProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -3128,12 +3128,12 @@ public final class MasterProcedureProtos {
         return regionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3151,22 +3151,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CreateTableStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CreateTableStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3178,7 +3178,7 @@ public final class MasterProcedureProtos {
 
   public interface ModifyTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyTableStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -3232,11 +3232,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.ModifyTableStateData}
    */
   public  static final class ModifyTableStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyTableStateData)
       ModifyTableStateDataOrBuilder {
     // Use ModifyTableStateData.newBuilder() to construct.
-    private ModifyTableStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyTableStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyTableStateData() {
@@ -3244,18 +3244,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyTableStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3317,22 +3317,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyTableStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3454,7 +3454,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -3477,19 +3477,19 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getUnmodifiedTableSchema());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getModifiedTableSchema());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, deleteColumnFamilyInModify_);
       }
       size += unknownFields.getSerializedSize();
@@ -3554,7 +3554,7 @@ public final class MasterProcedureProtos {
       }
       if (hasDeleteColumnFamilyInModify()) {
         hash = (37 * hash) + DELETE_COLUMN_FAMILY_IN_MODIFY_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDeleteColumnFamilyInModify());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -3563,61 +3563,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3635,7 +3635,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3643,15 +3643,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.ModifyTableStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyTableStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyTableStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3664,12 +3664,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getUnmodifiedTableSchemaFieldBuilder();
@@ -3701,7 +3701,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyTableStateData_descriptor;
       }
@@ -3759,29 +3759,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData)other);
         } else {
@@ -3834,13 +3834,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3853,7 +3853,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -3956,11 +3956,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -3971,7 +3971,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 2;
@@ -4074,11 +4074,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getUnmodifiedTableSchemaFieldBuilder() {
         if (unmodifiedTableSchemaBuilder_ == null) {
-          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          unmodifiedTableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getUnmodifiedTableSchema(),
                   getParentForChildren(),
@@ -4089,7 +4089,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema modifiedTableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> modifiedTableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema modified_table_schema = 3;
@@ -4192,11 +4192,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableSchema modified_table_schema = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getModifiedTableSchemaFieldBuilder() {
         if (modifiedTableSchemaBuilder_ == null) {
-          modifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          modifiedTableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getModifiedTableSchema(),
                   getParentForChildren(),
@@ -4238,12 +4238,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4261,22 +4261,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyTableStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyTableStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4288,7 +4288,7 @@ public final class MasterProcedureProtos {
 
   public interface TruncateTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TruncateTableStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -4366,11 +4366,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.TruncateTableStateData}
    */
   public  static final class TruncateTableStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TruncateTableStateData)
       TruncateTableStateDataOrBuilder {
     // Use TruncateTableStateData.newBuilder() to construct.
-    private TruncateTableStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TruncateTableStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TruncateTableStateData() {
@@ -4379,18 +4379,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TruncateTableStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4461,10 +4461,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
@@ -4474,12 +4474,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_TruncateTableStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_TruncateTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4640,7 +4640,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -4666,23 +4666,23 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, preserveSplits_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getTableName());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getTableSchema());
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, regionInfo_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -4741,7 +4741,7 @@ public final class MasterProcedureProtos {
       }
       if (hasPreserveSplits()) {
         hash = (37 * hash) + PRESERVE_SPLITS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getPreserveSplits());
       }
       if (hasTableName()) {
@@ -4762,61 +4762,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4834,7 +4834,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4842,15 +4842,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.TruncateTableStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TruncateTableStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_TruncateTableStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_TruncateTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4863,12 +4863,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -4907,7 +4907,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_TruncateTableStateData_descriptor;
       }
@@ -4974,29 +4974,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData)other);
         } else {
@@ -5038,7 +5038,7 @@ public final class MasterProcedureProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000010);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -5079,13 +5079,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TruncateTableStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5098,7 +5098,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -5201,11 +5201,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -5248,7 +5248,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * optional .hbase.pb.TableName table_name = 3;
@@ -5351,11 +5351,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.TableName table_name = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -5366,7 +5366,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
        * optional .hbase.pb.TableSchema table_schema = 4;
@@ -5469,11 +5469,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.TableSchema table_schema = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTableSchema(),
                   getParentForChildren(),
@@ -5492,7 +5492,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -5624,7 +5624,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -5708,11 +5708,11 @@ public final class MasterProcedureProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -5723,12 +5723,12 @@ public final class MasterProcedureProtos {
         return regionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5746,22 +5746,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TruncateTableStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TruncateTableStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5773,7 +5773,7 @@ public final class MasterProcedureProtos {
 
   public interface DeleteTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteTableStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -5829,11 +5829,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.DeleteTableStateData}
    */
   public  static final class DeleteTableStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteTableStateData)
       DeleteTableStateDataOrBuilder {
     // Use DeleteTableStateData.newBuilder() to construct.
-    private DeleteTableStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteTableStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteTableStateData() {
@@ -5841,18 +5841,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteTableStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5905,10 +5905,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -5918,12 +5918,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteTableStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6040,7 +6040,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -6060,15 +6060,15 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, regionInfo_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -6129,61 +6129,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6201,7 +6201,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6209,15 +6209,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.DeleteTableStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteTableStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteTableStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6230,12 +6230,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -6265,7 +6265,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteTableStateData_descriptor;
       }
@@ -6320,29 +6320,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData)other);
         } else {
@@ -6378,7 +6378,7 @@ public final class MasterProcedureProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000004);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -6412,13 +6412,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6431,7 +6431,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -6534,11 +6534,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -6549,7 +6549,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -6652,11 +6652,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -6675,7 +6675,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -6807,7 +6807,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -6891,11 +6891,11 @@ public final class MasterProcedureProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -6906,12 +6906,12 @@ public final class MasterProcedureProtos {
         return regionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6929,22 +6929,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteTableStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteTableStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6956,7 +6956,7 @@ public final class MasterProcedureProtos {
 
   public interface CreateNamespaceStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateNamespaceStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
@@ -6975,29 +6975,29 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.CreateNamespaceStateData}
    */
   public  static final class CreateNamespaceStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CreateNamespaceStateData)
       CreateNamespaceStateDataOrBuilder {
     // Use CreateNamespaceStateData.newBuilder() to construct.
-    private CreateNamespaceStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CreateNamespaceStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CreateNamespaceStateData() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CreateNamespaceStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7028,22 +7028,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7090,7 +7090,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getNamespaceDescriptor());
@@ -7104,7 +7104,7 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getNamespaceDescriptor());
       }
       size += unknownFields.getSerializedSize();
@@ -7150,61 +7150,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7222,7 +7222,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7230,15 +7230,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.CreateNamespaceStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CreateNamespaceStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7251,12 +7251,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
         }
@@ -7272,7 +7272,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CreateNamespaceStateData_descriptor;
       }
@@ -7310,29 +7310,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData)other);
         } else {
@@ -7362,13 +7362,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7381,7 +7381,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
       /**
        * required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
@@ -7484,11 +7484,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getNamespaceDescriptor(),
                   getParentForChildren(),
@@ -7498,12 +7498,12 @@ public final class MasterProcedureProtos {
         return namespaceDescriptorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7521,22 +7521,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CreateNamespaceStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CreateNamespaceStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7548,7 +7548,7 @@ public final class MasterProcedureProtos {
 
   public interface ModifyNamespaceStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyNamespaceStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
@@ -7580,29 +7580,29 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.ModifyNamespaceStateData}
    */
   public  static final class ModifyNamespaceStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyNamespaceStateData)
       ModifyNamespaceStateDataOrBuilder {
     // Use ModifyNamespaceStateData.newBuilder() to construct.
-    private ModifyNamespaceStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyNamespaceStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyNamespaceStateData() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyNamespaceStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7646,22 +7646,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7735,7 +7735,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getNamespaceDescriptor());
@@ -7752,11 +7752,11 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getNamespaceDescriptor());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getUnmodifiedNamespaceDescriptor());
       }
       size += unknownFields.getSerializedSize();
@@ -7811,61 +7811,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7883,7 +7883,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7891,15 +7891,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.ModifyNamespaceStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyNamespaceStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7912,12 +7912,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
           getUnmodifiedNamespaceDescriptorFieldBuilder();
@@ -7940,7 +7940,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyNamespaceStateData_descriptor;
       }
@@ -7986,29 +7986,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData)other);
         } else {
@@ -8046,13 +8046,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8065,7 +8065,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
       /**
        * required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
@@ -8168,11 +8168,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.NamespaceDescriptor namespace_descriptor = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getNamespaceDescriptor(),
                   getParentForChildren(),
@@ -8183,7 +8183,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor unmodifiedNamespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> unmodifiedNamespaceDescriptorBuilder_;
       /**
        * optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;
@@ -8286,11 +8286,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.NamespaceDescriptor unmodified_namespace_descriptor = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getUnmodifiedNamespaceDescriptorFieldBuilder() {
         if (unmodifiedNamespaceDescriptorBuilder_ == null) {
-          unmodifiedNamespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          unmodifiedNamespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getUnmodifiedNamespaceDescriptor(),
                   getParentForChildren(),
@@ -8300,12 +8300,12 @@ public final class MasterProcedureProtos {
         return unmodifiedNamespaceDescriptorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8323,22 +8323,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyNamespaceStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyNamespaceStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8350,7 +8350,7 @@ public final class MasterProcedureProtos {
 
   public interface DeleteNamespaceStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteNamespaceStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string namespace_name = 1;
@@ -8363,7 +8363,7 @@ public final class MasterProcedureProtos {
     /**
      * required string namespace_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes();
 
     /**
@@ -8383,11 +8383,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.DeleteNamespaceStateData}
    */
   public  static final class DeleteNamespaceStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteNamespaceStateData)
       DeleteNamespaceStateDataOrBuilder {
     // Use DeleteNamespaceStateData.newBuilder() to construct.
-    private DeleteNamespaceStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteNamespaceStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteNamespaceStateData() {
@@ -8395,18 +8395,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteNamespaceStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8423,7 +8423,7 @@ public final class MasterProcedureProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               namespaceName_ = bs;
               break;
@@ -8443,22 +8443,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteNamespaceStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteNamespaceStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8482,8 +8482,8 @@ public final class MasterProcedureProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespaceName_ = s;
@@ -8494,17 +8494,17 @@ public final class MasterProcedureProtos {
     /**
      * required string namespace_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes() {
       java.lang.Object ref = namespaceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespaceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -8549,10 +8549,10 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeMessage(2, getNamespaceDescriptor());
@@ -8566,10 +8566,10 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getNamespaceDescriptor());
       }
       size += unknownFields.getSerializedSize();
@@ -8624,61 +8624,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8696,7 +8696,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8704,15 +8704,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.DeleteNamespaceStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteNamespaceStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteNamespaceStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteNamespaceStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8725,12 +8725,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
         }
@@ -8748,7 +8748,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteNamespaceStateData_descriptor;
       }
@@ -8790,29 +8790,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData)other);
         } else {
@@ -8849,13 +8849,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8880,8 +8880,8 @@ public final class MasterProcedureProtos {
       public java.lang.String getNamespaceName() {
         java.lang.Object ref = namespaceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespaceName_ = s;
@@ -8894,17 +8894,17 @@ public final class MasterProcedureProtos {
       /**
        * required string namespace_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceNameBytes() {
         java.lang.Object ref = namespaceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespaceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -8933,7 +8933,7 @@ public final class MasterProcedureProtos {
        * required string namespace_name = 1;
        */
       public Builder setNamespaceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8944,7 +8944,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
       /**
        * optional .hbase.pb.NamespaceDescriptor namespace_descriptor = 2;
@@ -9047,11 +9047,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.NamespaceDescriptor namespace_descriptor = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getNamespaceDescriptor(),
                   getParentForChildren(),
@@ -9061,12 +9061,12 @@ public final class MasterProcedureProtos {
         return namespaceDescriptorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9084,22 +9084,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteNamespaceStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteNamespaceStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9111,7 +9111,7 @@ public final class MasterProcedureProtos {
 
   public interface AddColumnFamilyStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AddColumnFamilyStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -9169,29 +9169,29 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.AddColumnFamilyStateData}
    */
   public  static final class AddColumnFamilyStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AddColumnFamilyStateData)
       AddColumnFamilyStateDataOrBuilder {
     // Use AddColumnFamilyStateData.newBuilder() to construct.
-    private AddColumnFamilyStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AddColumnFamilyStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AddColumnFamilyStateData() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AddColumnFamilyStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9261,22 +9261,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_AddColumnFamilyStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_AddColumnFamilyStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9408,7 +9408,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -9431,19 +9431,19 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getColumnfamilySchema());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getUnmodifiedTableSchema());
       }
       size += unknownFields.getSerializedSize();
@@ -9516,61 +9516,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9588,7 +9588,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9596,15 +9596,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.AddColumnFamilyStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AddColumnFamilyStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_AddColumnFamilyStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_AddColumnFamilyStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9617,12 +9617,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -9659,7 +9659,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_AddColumnFamilyStateData_descriptor;
       }
@@ -9721,29 +9721,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData)other);
         } else {
@@ -9799,13 +9799,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9818,7 +9818,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -9921,11 +9921,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -9936,7 +9936,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -10039,11 +10039,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -10054,7 +10054,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema columnfamilySchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnfamilySchemaBuilder_;
       /**
        * required .hbase.pb.ColumnFamilySchema columnfamily_schema = 3;
@@ -10157,11 +10157,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.ColumnFamilySchema columnfamily_schema = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
           getColumnfamilySchemaFieldBuilder() {
         if (columnfamilySchemaBuilder_ == null) {
-          columnfamilySchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          columnfamilySchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
                   getColumnfamilySchema(),
                   getParentForChildren(),
@@ -10172,7 +10172,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
@@ -10275,11 +10275,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getUnmodifiedTableSchemaFieldBuilder() {
         if (unmodifiedTableSchemaBuilder_ == null) {
-          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          unmodifiedTableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getUnmodifiedTableSchema(),
                   getParentForChildren(),
@@ -10289,12 +10289,12 @@ public final class MasterProcedureProtos {
         return unmodifiedTableSchemaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10312,22 +10312,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AddColumnFamilyStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AddColumnFamilyStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10339,7 +10339,7 @@ public final class MasterProcedureProtos {
 
   public interface ModifyColumnFamilyStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyColumnFamilyStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -10397,29 +10397,29 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.ModifyColumnFamilyStateData}
    */
   public  static final class ModifyColumnFamilyStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyColumnFamilyStateData)
       ModifyColumnFamilyStateDataOrBuilder {
     // Use ModifyColumnFamilyStateData.newBuilder() to construct.
-    private ModifyColumnFamilyStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyColumnFamilyStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyColumnFamilyStateData() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyColumnFamilyStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10489,22 +10489,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyColumnFamilyStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10636,7 +10636,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -10659,19 +10659,19 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getColumnfamilySchema());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getUnmodifiedTableSchema());
       }
       size += unknownFields.getSerializedSize();
@@ -10744,61 +10744,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10816,7 +10816,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10824,15 +10824,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.ModifyColumnFamilyStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyColumnFamilyStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyColumnFamilyStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10845,12 +10845,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -10887,7 +10887,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor;
       }
@@ -10949,29 +10949,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData)other);
         } else {
@@ -11027,13 +11027,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11046,7 +11046,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -11149,11 +11149,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -11164,7 +11164,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -11267,11 +11267,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -11282,7 +11282,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema columnfamilySchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnfamilySchemaBuilder_;
       /**
        * required .hbase.pb.ColumnFamilySchema columnfamily_schema = 3;
@@ -11385,11 +11385,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.ColumnFamilySchema columnfamily_schema = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
           getColumnfamilySchemaFieldBuilder() {
         if (columnfamilySchemaBuilder_ == null) {
-          columnfamilySchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          columnfamilySchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
                   getColumnfamilySchema(),
                   getParentForChildren(),
@@ -11400,7 +11400,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
@@ -11503,11 +11503,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getUnmodifiedTableSchemaFieldBuilder() {
         if (unmodifiedTableSchemaBuilder_ == null) {
-          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          unmodifiedTableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getUnmodifiedTableSchema(),
                   getParentForChildren(),
@@ -11517,12 +11517,12 @@ public final class MasterProcedureProtos {
         return unmodifiedTableSchemaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11540,22 +11540,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyColumnFamilyStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyColumnFamilyStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11567,7 +11567,7 @@ public final class MasterProcedureProtos {
 
   public interface DeleteColumnFamilyStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteColumnFamilyStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -11602,7 +11602,7 @@ public final class MasterProcedureProtos {
     /**
      * required bytes columnfamily_name = 3;
      */
-    com.google.protobuf.ByteString getColumnfamilyName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnfamilyName();
 
     /**
      * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
@@ -11621,30 +11621,30 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.DeleteColumnFamilyStateData}
    */
   public  static final class DeleteColumnFamilyStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteColumnFamilyStateData)
       DeleteColumnFamilyStateDataOrBuilder {
     // Use DeleteColumnFamilyStateData.newBuilder() to construct.
-    private DeleteColumnFamilyStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteColumnFamilyStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteColumnFamilyStateData() {
-      columnfamilyName_ = com.google.protobuf.ByteString.EMPTY;
+      columnfamilyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteColumnFamilyStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11706,22 +11706,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteColumnFamilyStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11772,7 +11772,7 @@ public final class MasterProcedureProtos {
     }
 
     public static final int COLUMNFAMILY_NAME_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString columnfamilyName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnfamilyName_;
     /**
      * required bytes columnfamily_name = 3;
      */
@@ -11782,7 +11782,7 @@ public final class MasterProcedureProtos {
     /**
      * required bytes columnfamily_name = 3;
      */
-    public com.google.protobuf.ByteString getColumnfamilyName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnfamilyName() {
       return columnfamilyName_;
     }
 
@@ -11843,7 +11843,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -11866,19 +11866,19 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, columnfamilyName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getUnmodifiedTableSchema());
       }
       size += unknownFields.getSerializedSize();
@@ -11951,61 +11951,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12023,7 +12023,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12031,15 +12031,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.DeleteColumnFamilyStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteColumnFamilyStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteColumnFamilyStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12052,12 +12052,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -12078,7 +12078,7 @@ public final class MasterProcedureProtos {
           tableNameBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
-        columnfamilyName_ = com.google.protobuf.ByteString.EMPTY;
+        columnfamilyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         if (unmodifiedTableSchemaBuilder_ == null) {
           unmodifiedTableSchema_ = null;
@@ -12089,7 +12089,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor;
       }
@@ -12147,29 +12147,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData)other);
         } else {
@@ -12222,13 +12222,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12241,7 +12241,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -12344,11 +12344,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -12359,7 +12359,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -12462,11 +12462,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -12476,7 +12476,7 @@ public final class MasterProcedureProtos {
         return tableNameBuilder_;
       }
 
-      private com.google.protobuf.ByteString columnfamilyName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnfamilyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes columnfamily_name = 3;
        */
@@ -12486,13 +12486,13 @@ public final class MasterProcedureProtos {
       /**
        * required bytes columnfamily_name = 3;
        */
-      public com.google.protobuf.ByteString getColumnfamilyName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnfamilyName() {
         return columnfamilyName_;
       }
       /**
        * required bytes columnfamily_name = 3;
        */
-      public Builder setColumnfamilyName(com.google.protobuf.ByteString value) {
+      public Builder setColumnfamilyName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -12512,7 +12512,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
@@ -12615,11 +12615,11 @@ public final class MasterProcedureProtos {
       /**
        * optional .hbase.pb.TableSchema unmodified_table_schema = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getUnmodifiedTableSchemaFieldBuilder() {
         if (unmodifiedTableSchemaBuilder_ == null) {
-          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          unmodifiedTableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getUnmodifiedTableSchema(),
                   getParentForChildren(),
@@ -12629,12 +12629,12 @@ public final class MasterProcedureProtos {
         return unmodifiedTableSchemaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12652,22 +12652,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteColumnFamilyStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteColumnFamilyStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12679,7 +12679,7 @@ public final class MasterProcedureProtos {
 
   public interface EnableTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.EnableTableStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -12720,11 +12720,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.EnableTableStateData}
    */
   public  static final class EnableTableStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.EnableTableStateData)
       EnableTableStateDataOrBuilder {
     // Use EnableTableStateData.newBuilder() to construct.
-    private EnableTableStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private EnableTableStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private EnableTableStateData() {
@@ -12732,18 +12732,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private EnableTableStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12792,22 +12792,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_EnableTableStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_EnableTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12902,7 +12902,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -12922,15 +12922,15 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, skipTableStateCheck_);
       }
       size += unknownFields.getSerializedSize();
@@ -12986,7 +12986,7 @@ public final class MasterProcedureProtos {
       }
       if (hasSkipTableStateCheck()) {
         hash = (37 * hash) + SKIP_TABLE_STATE_CHECK_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getSkipTableStateCheck());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -12995,61 +12995,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -13067,7 +13067,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -13075,15 +13075,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.EnableTableStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.EnableTableStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_EnableTableStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_EnableTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13096,12 +13096,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -13126,7 +13126,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_EnableTableStateData_descriptor;
       }
@@ -13176,29 +13176,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData)other);
         } else {
@@ -13243,13 +13243,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13262,7 +13262,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -13365,11 +13365,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -13380,7 +13380,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -13483,11 +13483,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -13529,12 +13529,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -13552,22 +13552,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public EnableTableStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new EnableTableStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13579,7 +13579,7 @@ public final class MasterProcedureProtos {
 
   public interface DisableTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DisableTableStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -13620,11 +13620,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.DisableTableStateData}
    */
   public  static final class DisableTableStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DisableTableStateData)
       DisableTableStateDataOrBuilder {
     // Use DisableTableStateData.newBuilder() to construct.
-    private DisableTableStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DisableTableStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DisableTableStateData() {
@@ -13632,18 +13632,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DisableTableStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -13692,22 +13692,22 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DisableTableStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DisableTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -13802,7 +13802,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -13822,15 +13822,15 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, skipTableStateCheck_);
       }
       size += unknownFields.getSerializedSize();
@@ -13886,7 +13886,7 @@ public final class MasterProcedureProtos {
       }
       if (hasSkipTableStateCheck()) {
         hash = (37 * hash) + SKIP_TABLE_STATE_CHECK_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getSkipTableStateCheck());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -13895,61 +13895,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -13967,7 +13967,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -13975,15 +13975,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.DisableTableStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DisableTableStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DisableTableStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DisableTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13996,12 +13996,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -14026,7 +14026,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DisableTableStateData_descriptor;
       }
@@ -14076,29 +14076,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData)other);
         } else {
@@ -14143,13 +14143,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -14162,7 +14162,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -14265,11 +14265,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -14280,7 +14280,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -14383,11 +14383,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -14429,12 +14429,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14452,22 +14452,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DisableTableStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DisableTableStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14479,7 +14479,7 @@ public final class MasterProcedureProtos {
 
   public interface RestoreParentToChildRegionsPairOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RestoreParentToChildRegionsPair)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string parent_region_name = 1;
@@ -14492,7 +14492,7 @@ public final class MasterProcedureProtos {
     /**
      * required string parent_region_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getParentRegionNameBytes();
 
     /**
@@ -14506,7 +14506,7 @@ public final class MasterProcedureProtos {
     /**
      * required string child1_region_name = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getChild1RegionNameBytes();
 
     /**
@@ -14520,18 +14520,18 @@ public final class MasterProcedureProtos {
     /**
      * required string child2_region_name = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getChild2RegionNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.RestoreParentToChildRegionsPair}
    */
   public  static final class RestoreParentToChildRegionsPair extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RestoreParentToChildRegionsPair)
       RestoreParentToChildRegionsPairOrBuilder {
     // Use RestoreParentToChildRegionsPair.newBuilder() to construct.
-    private RestoreParentToChildRegionsPair(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RestoreParentToChildRegionsPair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RestoreParentToChildRegionsPair() {
@@ -14541,18 +14541,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RestoreParentToChildRegionsPair(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14569,41 +14569,41 @@ public final class MasterProcedureProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               parentRegionName_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               child1RegionName_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               child2RegionName_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreParentToChildRegionsPair_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14627,8 +14627,8 @@ public final class MasterProcedureProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           parentRegionName_ = s;
@@ -14639,17 +14639,17 @@ public final class MasterProcedureProtos {
     /**
      * required string parent_region_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getParentRegionNameBytes() {
       java.lang.Object ref = parentRegionName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         parentRegionName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -14669,8 +14669,8 @@ public final class MasterProcedureProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           child1RegionName_ = s;
@@ -14681,17 +14681,17 @@ public final class MasterProcedureProtos {
     /**
      * required string child1_region_name = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getChild1RegionNameBytes() {
       java.lang.Object ref = child1RegionName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         child1RegionName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -14711,8 +14711,8 @@ public final class MasterProcedureProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           child2RegionName_ = s;
@@ -14723,17 +14723,17 @@ public final class MasterProcedureProtos {
     /**
      * required string child2_region_name = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getChild2RegionNameBytes() {
       java.lang.Object ref = child2RegionName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         child2RegionName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -14759,16 +14759,16 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, parentRegionName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, parentRegionName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, child1RegionName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, child1RegionName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, child2RegionName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, child2RegionName_);
       }
       unknownFields.writeTo(output);
     }
@@ -14779,13 +14779,13 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, parentRegionName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, parentRegionName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, child1RegionName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, child1RegionName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, child2RegionName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, child2RegionName_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -14848,61 +14848,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14920,7 +14920,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -14928,15 +14928,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.RestoreParentToChildRegionsPair}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RestoreParentToChildRegionsPair)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreParentToChildRegionsPair_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -14949,12 +14949,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -14969,7 +14969,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor;
       }
@@ -15011,29 +15011,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair)other);
         } else {
@@ -15078,13 +15078,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15109,8 +15109,8 @@ public final class MasterProcedureProtos {
       public java.lang.String getParentRegionName() {
         java.lang.Object ref = parentRegionName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             parentRegionName_ = s;
@@ -15123,17 +15123,17 @@ public final class MasterProcedureProtos {
       /**
        * required string parent_region_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getParentRegionNameBytes() {
         java.lang.Object ref = parentRegionName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           parentRegionName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -15162,7 +15162,7 @@ public final class MasterProcedureProtos {
        * required string parent_region_name = 1;
        */
       public Builder setParentRegionNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -15185,8 +15185,8 @@ public final class MasterProcedureProtos {
       public java.lang.String getChild1RegionName() {
         java.lang.Object ref = child1RegionName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             child1RegionName_ = s;
@@ -15199,17 +15199,17 @@ public final class MasterProcedureProtos {
       /**
        * required string child1_region_name = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getChild1RegionNameBytes() {
         java.lang.Object ref = child1RegionName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           child1RegionName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -15238,7 +15238,7 @@ public final class MasterProcedureProtos {
        * required string child1_region_name = 2;
        */
       public Builder setChild1RegionNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -15261,8 +15261,8 @@ public final class MasterProcedureProtos {
       public java.lang.String getChild2RegionName() {
         java.lang.Object ref = child2RegionName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             child2RegionName_ = s;
@@ -15275,17 +15275,17 @@ public final class MasterProcedureProtos {
       /**
        * required string child2_region_name = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getChild2RegionNameBytes() {
         java.lang.Object ref = child2RegionName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           child2RegionName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -15314,7 +15314,7 @@ public final class MasterProcedureProtos {
        * required string child2_region_name = 3;
        */
       public Builder setChild2RegionNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -15324,12 +15324,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15347,22 +15347,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RestoreParentToChildRegionsPair parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RestoreParentToChildRegionsPair(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15374,7 +15374,7 @@ public final class MasterProcedureProtos {
 
   public interface CloneSnapshotStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CloneSnapshotStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -15467,11 +15467,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.CloneSnapshotStateData}
    */
   public  static final class CloneSnapshotStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CloneSnapshotStateData)
       CloneSnapshotStateDataOrBuilder {
     // Use CloneSnapshotStateData.newBuilder() to construct.
-    private CloneSnapshotStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CloneSnapshotStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CloneSnapshotStateData() {
@@ -15480,18 +15480,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CloneSnapshotStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15566,10 +15566,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
@@ -15582,12 +15582,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CloneSnapshotStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CloneSnapshotStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -15774,7 +15774,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -15800,23 +15800,23 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getSnapshot());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getTableSchema());
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, regionInfo_.get(i));
       }
       for (int i = 0; i < parentToChildRegionsPairList_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, parentToChildRegionsPairList_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -15892,61 +15892,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15964,7 +15964,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15972,15 +15972,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.CloneSnapshotStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CloneSnapshotStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CloneSnapshotStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CloneSnapshotStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15993,12 +15993,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getSnapshotFieldBuilder();
@@ -16042,7 +16042,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_CloneSnapshotStateData_descriptor;
       }
@@ -16114,29 +16114,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData)other);
         } else {
@@ -16175,7 +16175,7 @@ public final class MasterProcedureProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000008);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -16201,7 +16201,7 @@ public final class MasterProcedureProtos {
               parentToChildRegionsPairList_ = other.parentToChildRegionsPairList_;
               bitField0_ = (bitField0_ & ~0x00000010);
               parentToChildRegionsPairListBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getParentToChildRegionsPairListFieldBuilder() : null;
             } else {
               parentToChildRegionsPairListBuilder_.addAllMessages(other.parentToChildRegionsPairList_);
@@ -16246,13 +16246,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16265,7 +16265,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -16368,11 +16368,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -16383,7 +16383,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 2;
@@ -16486,11 +16486,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -16501,7 +16501,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema table_schema = 3;
@@ -16604,11 +16604,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableSchema table_schema = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTableSchema(),
                   getParentForChildren(),
@@ -16627,7 +16627,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -16759,7 +16759,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -16843,11 +16843,11 @@ public final class MasterProcedureProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -16867,7 +16867,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder> parentToChildRegionsPairListBuilder_;
 
       /**
@@ -16999,7 +16999,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (parentToChildRegionsPairListBuilder_ == null) {
           ensureParentToChildRegionsPairListIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, parentToChildRegionsPairList_);
           onChanged();
         } else {
@@ -17083,11 +17083,11 @@ public final class MasterProcedureProtos {
            getParentToChildRegionsPairListBuilderList() {
         return getParentToChildRegionsPairListFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder> 
           getParentToChildRegionsPairListFieldBuilder() {
         if (parentToChildRegionsPairListBuilder_ == null) {
-          parentToChildRegionsPairListBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          parentToChildRegionsPairListBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder>(
                   parentToChildRegionsPairList_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -17098,12 +17098,12 @@ public final class MasterProcedureProtos {
         return parentToChildRegionsPairListBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17121,22 +17121,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CloneSnapshotStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CloneSnapshotStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17148,7 +17148,7 @@ public final class MasterProcedureProtos {
 
   public interface RestoreSnapshotStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RestoreSnapshotStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -17289,11 +17289,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.RestoreSnapshotStateData}
    */
   public  static final class RestoreSnapshotStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RestoreSnapshotStateData)
       RestoreSnapshotStateDataOrBuilder {
     // Use RestoreSnapshotStateData.newBuilder() to construct.
-    private RestoreSnapshotStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RestoreSnapshotStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RestoreSnapshotStateData() {
@@ -17304,18 +17304,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RestoreSnapshotStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -17408,10 +17408,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
@@ -17430,12 +17430,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreSnapshotStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreSnapshotStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -17704,7 +17704,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -17736,31 +17736,31 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getSnapshot());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getModifiedTableSchema());
       }
       for (int i = 0; i < regionInfoForRestore_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, regionInfoForRestore_.get(i));
       }
       for (int i = 0; i < regionInfoForRemove_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, regionInfoForRemove_.get(i));
       }
       for (int i = 0; i < regionInfoForAdd_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, regionInfoForAdd_.get(i));
       }
       for (int i = 0; i < parentToChildRegionsPairList_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(7, parentToChildRegionsPairList_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -17848,61 +17848,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -17920,7 +17920,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -17928,15 +17928,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.RestoreSnapshotStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RestoreSnapshotStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreSnapshotStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreSnapshotStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -17949,12 +17949,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getSnapshotFieldBuilder();
@@ -18012,7 +18012,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_RestoreSnapshotStateData_descriptor;
       }
@@ -18102,29 +18102,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData)other);
         } else {
@@ -18163,7 +18163,7 @@ public final class MasterProcedureProtos {
               regionInfoForRestore_ = other.regionInfoForRestore_;
               bitField0_ = (bitField0_ & ~0x00000008);
               regionInfoForRestoreBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoForRestoreFieldBuilder() : null;
             } else {
               regionInfoForRestoreBuilder_.addAllMessages(other.regionInfoForRestore_);
@@ -18189,7 +18189,7 @@ public final class MasterProcedureProtos {
               regionInfoForRemove_ = other.regionInfoForRemove_;
               bitField0_ = (bitField0_ & ~0x00000010);
               regionInfoForRemoveBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoForRemoveFieldBuilder() : null;
             } else {
               regionInfoForRemoveBuilder_.addAllMessages(other.regionInfoForRemove_);
@@ -18215,7 +18215,7 @@ public final class MasterProcedureProtos {
               regionInfoForAdd_ = other.regionInfoForAdd_;
               bitField0_ = (bitField0_ & ~0x00000020);
               regionInfoForAddBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoForAddFieldBuilder() : null;
             } else {
               regionInfoForAddBuilder_.addAllMessages(other.regionInfoForAdd_);
@@ -18241,7 +18241,7 @@ public final class MasterProcedureProtos {
               parentToChildRegionsPairList_ = other.parentToChildRegionsPairList_;
               bitField0_ = (bitField0_ & ~0x00000040);
               parentToChildRegionsPairListBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getParentToChildRegionsPairListFieldBuilder() : null;
             } else {
               parentToChildRegionsPairListBuilder_.addAllMessages(other.parentToChildRegionsPairList_);
@@ -18296,13 +18296,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -18315,7 +18315,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -18418,11 +18418,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -18433,7 +18433,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 2;
@@ -18536,11 +18536,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -18551,7 +18551,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema modifiedTableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> modifiedTableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema modified_table_schema = 3;
@@ -18654,11 +18654,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableSchema modified_table_schema = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getModifiedTableSchemaFieldBuilder() {
         if (modifiedTableSchemaBuilder_ == null) {
-          modifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          modifiedTableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getModifiedTableSchema(),
                   getParentForChildren(),
@@ -18677,7 +18677,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoForRestoreBuilder_;
 
       /**
@@ -18809,7 +18809,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoForRestoreBuilder_ == null) {
           ensureRegionInfoForRestoreIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfoForRestore_);
           onChanged();
         } else {
@@ -18893,11 +18893,11 @@ public final class MasterProcedureProtos {
            getRegionInfoForRestoreBuilderList() {
         return getRegionInfoForRestoreFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoForRestoreFieldBuilder() {
         if (regionInfoForRestoreBuilder_ == null) {
-          regionInfoForRestoreBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoForRestoreBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfoForRestore_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -18917,7 +18917,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoForRemoveBuilder_;
 
       /**
@@ -19049,7 +19049,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoForRemoveBuilder_ == null) {
           ensureRegionInfoForRemoveIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfoForRemove_);
           onChanged();
         } else {
@@ -19133,11 +19133,11 @@ public final class MasterProcedureProtos {
            getRegionInfoForRemoveBuilderList() {
         return getRegionInfoForRemoveFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoForRemoveFieldBuilder() {
         if (regionInfoForRemoveBuilder_ == null) {
-          regionInfoForRemoveBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoForRemoveBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfoForRemove_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -19157,7 +19157,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoForAddBuilder_;
 
       /**
@@ -19289,7 +19289,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoForAddBuilder_ == null) {
           ensureRegionInfoForAddIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfoForAdd_);
           onChanged();
         } else {
@@ -19373,11 +19373,11 @@ public final class MasterProcedureProtos {
            getRegionInfoForAddBuilderList() {
         return getRegionInfoForAddFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoForAddFieldBuilder() {
         if (regionInfoForAddBuilder_ == null) {
-          regionInfoForAddBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoForAddBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfoForAdd_,
                   ((bitField0_ & 0x00000020) == 0x00000020),
@@ -19397,7 +19397,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder> parentToChildRegionsPairListBuilder_;
 
       /**
@@ -19529,7 +19529,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (parentToChildRegionsPairListBuilder_ == null) {
           ensureParentToChildRegionsPairListIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, parentToChildRegionsPairList_);
           onChanged();
         } else {
@@ -19613,11 +19613,11 @@ public final class MasterProcedureProtos {
            getParentToChildRegionsPairListBuilderList() {
         return getParentToChildRegionsPairListFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder> 
           getParentToChildRegionsPairListFieldBuilder() {
         if (parentToChildRegionsPairListBuilder_ == null) {
-          parentToChildRegionsPairListBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          parentToChildRegionsPairListBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreParentToChildRegionsPairOrBuilder>(
                   parentToChildRegionsPairList_,
                   ((bitField0_ & 0x00000040) == 0x00000040),
@@ -19628,12 +19628,12 @@ public final class MasterProcedureProtos {
         return parentToChildRegionsPairListBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -19651,22 +19651,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RestoreSnapshotStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RestoreSnapshotStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -19678,7 +19678,7 @@ public final class MasterProcedureProtos {
 
   public interface DispatchMergingRegionsStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.UserInformation user_info = 1;
@@ -19743,11 +19743,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
    */
   public  static final class DispatchMergingRegionsStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DispatchMergingRegionsStateData)
       DispatchMergingRegionsStateDataOrBuilder {
     // Use DispatchMergingRegionsStateData.newBuilder() to construct.
-    private DispatchMergingRegionsStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DispatchMergingRegionsStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DispatchMergingRegionsStateData() {
@@ -19756,18 +19756,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DispatchMergingRegionsStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19825,10 +19825,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -19838,12 +19838,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19975,7 +19975,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
@@ -19998,19 +19998,19 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableName());
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, regionInfo_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, forcible_);
       }
       size += unknownFields.getSerializedSize();
@@ -20072,7 +20072,7 @@ public final class MasterProcedureProtos {
       }
       if (hasForcible()) {
         hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getForcible());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -20081,61 +20081,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -20153,7 +20153,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -20161,15 +20161,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DispatchMergingRegionsStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -20182,12 +20182,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getTableNameFieldBuilder();
@@ -20219,7 +20219,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
       }
@@ -20278,29 +20278,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)other);
         } else {
@@ -20336,7 +20336,7 @@ public final class MasterProcedureProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000004);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -20373,13 +20373,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20392,7 +20392,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * required .hbase.pb.UserInformation user_info = 1;
@@ -20495,11 +20495,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -20510,7 +20510,7 @@ public final class MasterProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 2;
@@ -20613,11 +20613,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.TableName table_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -20636,7 +20636,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -20768,7 +20768,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -20852,11 +20852,11 @@ public final class MasterProcedureProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -20899,12 +20899,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -20922,22 +20922,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DispatchMergingRegionsStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DispatchMergingRegionsStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -20949,7 +20949,7 @@ public final class MasterProcedureProtos {
 
   public interface ServerCrashStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ServerCrashStateData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ServerName server_name = 1;
@@ -21043,11 +21043,11 @@ public final class MasterProcedureProtos {
    * Protobuf type {@code hbase.pb.ServerCrashStateData}
    */
   public  static final class ServerCrashStateData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ServerCrashStateData)
       ServerCrashStateDataOrBuilder {
     // Use ServerCrashStateData.newBuilder() to construct.
-    private ServerCrashStateData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ServerCrashStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ServerCrashStateData() {
@@ -21059,18 +21059,18 @@ public final class MasterProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ServerCrashStateData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -21134,10 +21134,10 @@ public final class MasterProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -21150,12 +21150,12 @@ public final class MasterProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -21329,7 +21329,7 @@ public final class MasterProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServerName());
@@ -21358,27 +21358,27 @@ public final class MasterProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServerName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, distributedLogReplay_);
       }
       for (int i = 0; i < regionsOnCrashedServer_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, regionsOnCrashedServer_.get(i));
       }
       for (int i = 0; i < regionsAssigned_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, regionsAssigned_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, carryingMeta_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(6, shouldSplitWal_);
       }
       size += unknownFields.getSerializedSize();
@@ -21439,7 +21439,7 @@ public final class MasterProcedureProtos {
       }
       if (hasDistributedLogReplay()) {
         hash = (37 * hash) + DISTRIBUTED_LOG_REPLAY_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDistributedLogReplay());
       }
       if (getRegionsOnCrashedServerCount() > 0) {
@@ -21452,12 +21452,12 @@ public final class MasterProcedureProtos {
       }
       if (hasCarryingMeta()) {
         hash = (37 * hash) + CARRYING_META_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getCarryingMeta());
       }
       if (hasShouldSplitWal()) {
         hash = (37 * hash) + SHOULD_SPLIT_WAL_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getShouldSplitWal());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -21466,61 +21466,61 @@ public final class MasterProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -21538,7 +21538,7 @@ public final class MasterProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -21546,15 +21546,15 @@ public final class MasterProcedureProtos {
      * Protobuf type {@code hbase.pb.ServerCrashStateData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ServerCrashStateData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21567,12 +21567,12 @@ public final class MasterProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerNameFieldBuilder();
           getRegionsOnCrashedServerFieldBuilder();
@@ -21608,7 +21608,7 @@ public final class MasterProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_ServerCrashStateData_descriptor;
       }
@@ -21676,29 +21676,29 @@ public final class MasterProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData)other);
         } else {
@@ -21734,7 +21734,7 @@ public final class MasterProcedureProtos {
               regionsOnCrashedServer_ = other.regionsOnCrashedServer_;
               bitField0_ = (bitField0_ & ~0x00000004);
               regionsOnCrashedServerBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionsOnCrashedServerFieldBuilder() : null;
             } else {
               regionsOnCrashedServerBuilder_.addAllMessages(other.regionsOnCrashedServer_);
@@ -21760,7 +21760,7 @@ public final class MasterProcedureProtos {
               regionsAssigned_ = other.regionsAssigned_;
               bitField0_ = (bitField0_ & ~0x00000008);
               regionsAssignedBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionsAssignedFieldBuilder() : null;
             } else {
               regionsAssignedBuilder_.addAllMessages(other.regionsAssigned_);
@@ -21799,13 +21799,13 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -21818,7 +21818,7 @@ public final class MasterProcedureProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_;
       /**
        * required .hbase.pb.ServerName server_name = 1;
@@ -21921,11 +21921,11 @@ public final class MasterProcedureProtos {
       /**
        * required .hbase.pb.ServerName server_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerNameFieldBuilder() {
         if (serverNameBuilder_ == null) {
-          serverNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServerName(),
                   getParentForChildren(),
@@ -21976,7 +21976,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionsOnCrashedServerBuilder_;
 
       /**
@@ -22108,7 +22108,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionsOnCrashedServerBuilder_ == null) {
           ensureRegionsOnCrashedServerIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionsOnCrashedServer_);
           onChanged();
         } else {
@@ -22192,11 +22192,11 @@ public final class MasterProcedureProtos {
            getRegionsOnCrashedServerBuilderList() {
         return getRegionsOnCrashedServerFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionsOnCrashedServerFieldBuilder() {
         if (regionsOnCrashedServerBuilder_ == null) {
-          regionsOnCrashedServerBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionsOnCrashedServerBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionsOnCrashedServer_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -22216,7 +22216,7 @@ public final class MasterProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionsAssignedBuilder_;
 
       /**
@@ -22348,7 +22348,7 @@ public final class MasterProcedureProtos {
           java.lang.Iterable values) {
         if (regionsAssignedBuilder_ == null) {
           ensureRegionsAssignedIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionsAssigned_);
           onChanged();
         } else {
@@ -22432,11 +22432,11 @@ public final class MasterProcedureProtos {
            getRegionsAssignedBuilderList() {
         return getRegionsAssignedFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionsAssignedFieldBuilder() {
         if (regionsAssignedBuilder_ == null) {
-          regionsAssignedBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionsAssignedBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionsAssigned_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -22511,12 +22511,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -22534,22 +22534,22 @@ public final class MasterProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ServerCrashStateData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ServerCrashStateData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -22559,97 +22559,97 @@ public final class MasterProcedureProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CreateTableStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CreateTableStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyTableStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyTableStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TruncateTableStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TruncateTableStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteTableStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteTableStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CreateNamespaceStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyNamespaceStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteNamespaceStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteNamespaceStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AddColumnFamilyStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AddColumnFamilyStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyColumnFamilyStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteColumnFamilyStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_EnableTableStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_EnableTableStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DisableTableStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DisableTableStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RestoreParentToChildRegionsPair_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CloneSnapshotStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CloneSnapshotStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RestoreSnapshotStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RestoreSnapshotStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ServerCrashStateData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -22835,120 +22835,120 @@ public final class MasterProcedureProtos {
       ".shaded.protobuf.generatedB\025MasterProced",
       "ureProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_CreateTableStateData_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_CreateTableStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CreateTableStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableSchema", "RegionInfo", });
     internal_static_hbase_pb_ModifyTableStateData_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_ModifyTableStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyTableStateData_descriptor,
         new java.lang.String[] { "UserInfo", "UnmodifiedTableSchema", "ModifiedTableSchema", "DeleteColumnFamilyInModify", });
     internal_static_hbase_pb_TruncateTableStateData_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_TruncateTableStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TruncateTableStateData_descriptor,
         new java.lang.String[] { "UserInfo", "PreserveSplits", "TableName", "TableSchema", "RegionInfo", });
     internal_static_hbase_pb_DeleteTableStateData_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_DeleteTableStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteTableStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "RegionInfo", });
     internal_static_hbase_pb_CreateNamespaceStateData_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CreateNamespaceStateData_descriptor,
         new java.lang.String[] { "NamespaceDescriptor", });
     internal_static_hbase_pb_ModifyNamespaceStateData_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyNamespaceStateData_descriptor,
         new java.lang.String[] { "NamespaceDescriptor", "UnmodifiedNamespaceDescriptor", });
     internal_static_hbase_pb_DeleteNamespaceStateData_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_DeleteNamespaceStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteNamespaceStateData_descriptor,
         new java.lang.String[] { "NamespaceName", "NamespaceDescriptor", });
     internal_static_hbase_pb_AddColumnFamilyStateData_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_AddColumnFamilyStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AddColumnFamilyStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "ColumnfamilySchema", "UnmodifiedTableSchema", });
     internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_ModifyColumnFamilyStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "ColumnfamilySchema", "UnmodifiedTableSchema", });
     internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_DeleteColumnFamilyStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "ColumnfamilyName", "UnmodifiedTableSchema", });
     internal_static_hbase_pb_EnableTableStateData_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_EnableTableStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_EnableTableStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "SkipTableStateCheck", });
     internal_static_hbase_pb_DisableTableStateData_descriptor =
       getDescriptor().getMessageTypes().get(11);
     internal_static_hbase_pb_DisableTableStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DisableTableStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "SkipTableStateCheck", });
     internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_RestoreParentToChildRegionsPair_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor,
         new java.lang.String[] { "ParentRegionName", "Child1RegionName", "Child2RegionName", });
     internal_static_hbase_pb_CloneSnapshotStateData_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_hbase_pb_CloneSnapshotStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CloneSnapshotStateData_descriptor,
         new java.lang.String[] { "UserInfo", "Snapshot", "TableSchema", "RegionInfo", "ParentToChildRegionsPairList", });
     internal_static_hbase_pb_RestoreSnapshotStateData_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_RestoreSnapshotStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RestoreSnapshotStateData_descriptor,
         new java.lang.String[] { "UserInfo", "Snapshot", "ModifiedTableSchema", "RegionInfoForRestore", "RegionInfoForRemove", "RegionInfoForAdd", "ParentToChildRegionsPairList", });
     internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor =
       getDescriptor().getMessageTypes().get(15);
     internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor,
         new java.lang.String[] { "UserInfo", "TableName", "RegionInfo", "Forcible", });
     internal_static_hbase_pb_ServerCrashStateData_descriptor =
       getDescriptor().getMessageTypes().get(16);
     internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ServerCrashStateData_descriptor,
         new java.lang.String[] { "ServerName", "DistributedLogReplay", "RegionsOnCrashedServer", "RegionsAssigned", "CarryingMeta", "ShouldSplitWal", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 4426013..03ef208 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -6,19 +6,19 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class MasterProtos {
   private MasterProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * Protobuf enum {@code hbase.pb.MasterSwitchType}
    */
   public enum MasterSwitchType
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * SPLIT = 0;
      */
@@ -59,27 +59,27 @@ public final class MasterProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         MasterSwitchType> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public MasterSwitchType findValueByNumber(int number) {
               return MasterSwitchType.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -87,7 +87,7 @@ public final class MasterProtos {
     private static final MasterSwitchType[] VALUES = values();
 
     public static MasterSwitchType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -106,7 +106,7 @@ public final class MasterProtos {
 
   public interface AddColumnRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AddColumnRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -156,11 +156,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.AddColumnRequest}
    */
   public  static final class AddColumnRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AddColumnRequest)
       AddColumnRequestOrBuilder {
     // Use AddColumnRequest.newBuilder() to construct.
-    private AddColumnRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AddColumnRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AddColumnRequest() {
@@ -169,18 +169,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AddColumnRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -234,22 +234,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -355,7 +355,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -378,19 +378,19 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getColumnFamilies());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonceGroup_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -451,12 +451,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -465,61 +465,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -537,7 +537,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -545,15 +545,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.AddColumnRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AddColumnRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -566,12 +566,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getColumnFamiliesFieldBuilder();
@@ -598,7 +598,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnRequest_descriptor;
       }
@@ -652,29 +652,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest)other);
         } else {
@@ -719,13 +719,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -738,7 +738,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -841,11 +841,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -856,7 +856,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema columnFamilies_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnFamiliesBuilder_;
       /**
        * required .hbase.pb.ColumnFamilySchema column_families = 2;
@@ -959,11 +959,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.ColumnFamilySchema column_families = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
           getColumnFamiliesFieldBuilder() {
         if (columnFamiliesBuilder_ == null) {
-          columnFamiliesBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          columnFamiliesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
                   getColumnFamilies(),
                   getParentForChildren(),
@@ -1037,12 +1037,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1060,22 +1060,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AddColumnRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AddColumnRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1087,7 +1087,7 @@ public final class MasterProtos {
 
   public interface AddColumnResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AddColumnResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -1102,11 +1102,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.AddColumnResponse}
    */
   public  static final class AddColumnResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AddColumnResponse)
       AddColumnResponseOrBuilder {
     // Use AddColumnResponse.newBuilder() to construct.
-    private AddColumnResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AddColumnResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AddColumnResponse() {
@@ -1114,18 +1114,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AddColumnResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1148,22 +1148,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1196,7 +1196,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -1210,7 +1210,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -1248,7 +1248,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -1257,61 +1257,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1329,7 +1329,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1337,15 +1337,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.AddColumnResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AddColumnResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1358,12 +1358,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -1374,7 +1374,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AddColumnResponse_descriptor;
       }
@@ -1408,29 +1408,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse)other);
         } else {
@@ -1454,13 +1454,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1504,12 +1504,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1527,22 +1527,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AddColumnResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AddColumnResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1554,7 +1554,7 @@ public final class MasterProtos {
 
   public interface DeleteColumnRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteColumnRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -1576,7 +1576,7 @@ public final class MasterProtos {
     /**
      * required bytes column_name = 2;
      */
-    com.google.protobuf.ByteString getColumnName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnName();
 
     /**
      * optional uint64 nonce_group = 3 [default = 0];
@@ -1600,32 +1600,32 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteColumnRequest}
    */
   public  static final class DeleteColumnRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteColumnRequest)
       DeleteColumnRequestOrBuilder {
     // Use DeleteColumnRequest.newBuilder() to construct.
-    private DeleteColumnRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteColumnRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteColumnRequest() {
-      columnName_ = com.google.protobuf.ByteString.EMPTY;
+      columnName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       nonceGroup_ = 0L;
       nonce_ = 0L;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteColumnRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1671,22 +1671,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1716,7 +1716,7 @@ public final class MasterProtos {
     }
 
     public static final int COLUMN_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString columnName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnName_;
     /**
      * required bytes column_name = 2;
      */
@@ -1726,7 +1726,7 @@ public final class MasterProtos {
     /**
      * required bytes column_name = 2;
      */
-    public com.google.protobuf.ByteString getColumnName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnName() {
       return columnName_;
     }
 
@@ -1782,7 +1782,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -1805,19 +1805,19 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, columnName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonceGroup_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -1878,12 +1878,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -1892,61 +1892,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1964,7 +1964,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1972,15 +1972,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteColumnRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteColumnRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1993,12 +1993,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -2011,7 +2011,7 @@ public final class MasterProtos {
           tableNameBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        columnName_ = com.google.protobuf.ByteString.EMPTY;
+        columnName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         nonceGroup_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -2020,7 +2020,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnRequest_descriptor;
       }
@@ -2070,29 +2070,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest)other);
         } else {
@@ -2134,13 +2134,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2153,7 +2153,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -2256,11 +2256,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -2270,7 +2270,7 @@ public final class MasterProtos {
         return tableNameBuilder_;
       }
 
-      private com.google.protobuf.ByteString columnName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString columnName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes column_name = 2;
        */
@@ -2280,13 +2280,13 @@ public final class MasterProtos {
       /**
        * required bytes column_name = 2;
        */
-      public com.google.protobuf.ByteString getColumnName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getColumnName() {
         return columnName_;
       }
       /**
        * required bytes column_name = 2;
        */
-      public Builder setColumnName(com.google.protobuf.ByteString value) {
+      public Builder setColumnName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2369,12 +2369,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2392,22 +2392,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteColumnRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteColumnRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2419,7 +2419,7 @@ public final class MasterProtos {
 
   public interface DeleteColumnResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteColumnResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -2434,11 +2434,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteColumnResponse}
    */
   public  static final class DeleteColumnResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteColumnResponse)
       DeleteColumnResponseOrBuilder {
     // Use DeleteColumnResponse.newBuilder() to construct.
-    private DeleteColumnResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteColumnResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteColumnResponse() {
@@ -2446,18 +2446,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteColumnResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2480,22 +2480,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2528,7 +2528,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -2542,7 +2542,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -2580,7 +2580,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -2589,61 +2589,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2661,7 +2661,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2669,15 +2669,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteColumnResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteColumnResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2690,12 +2690,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -2706,7 +2706,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteColumnResponse_descriptor;
       }
@@ -2740,29 +2740,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse)other);
         } else {
@@ -2786,13 +2786,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2836,12 +2836,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2859,22 +2859,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteColumnResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteColumnResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2886,7 +2886,7 @@ public final class MasterProtos {
 
   public interface ModifyColumnRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyColumnRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -2936,11 +2936,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ModifyColumnRequest}
    */
   public  static final class ModifyColumnRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyColumnRequest)
       ModifyColumnRequestOrBuilder {
     // Use ModifyColumnRequest.newBuilder() to construct.
-    private ModifyColumnRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyColumnRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyColumnRequest() {
@@ -2949,18 +2949,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyColumnRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3014,22 +3014,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3135,7 +3135,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -3158,19 +3158,19 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getColumnFamilies());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonceGroup_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -3231,12 +3231,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -3245,61 +3245,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3317,7 +3317,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3325,15 +3325,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ModifyColumnRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyColumnRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3346,12 +3346,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getColumnFamiliesFieldBuilder();
@@ -3378,7 +3378,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnRequest_descriptor;
       }
@@ -3432,29 +3432,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest)other);
         } else {
@@ -3499,13 +3499,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3518,7 +3518,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -3621,11 +3621,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -3636,7 +3636,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema columnFamilies_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnFamiliesBuilder_;
       /**
        * required .hbase.pb.ColumnFamilySchema column_families = 2;
@@ -3739,11 +3739,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.ColumnFamilySchema column_families = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
           getColumnFamiliesFieldBuilder() {
         if (columnFamiliesBuilder_ == null) {
-          columnFamiliesBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          columnFamiliesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
                   getColumnFamilies(),
                   getParentForChildren(),
@@ -3817,12 +3817,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3840,22 +3840,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyColumnRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyColumnRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3867,7 +3867,7 @@ public final class MasterProtos {
 
   public interface ModifyColumnResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyColumnResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -3882,11 +3882,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ModifyColumnResponse}
    */
   public  static final class ModifyColumnResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyColumnResponse)
       ModifyColumnResponseOrBuilder {
     // Use ModifyColumnResponse.newBuilder() to construct.
-    private ModifyColumnResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyColumnResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyColumnResponse() {
@@ -3894,18 +3894,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyColumnResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3928,22 +3928,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3976,7 +3976,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -3990,7 +3990,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -4028,7 +4028,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -4037,61 +4037,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4109,7 +4109,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4117,15 +4117,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ModifyColumnResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyColumnResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4138,12 +4138,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -4154,7 +4154,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyColumnResponse_descriptor;
       }
@@ -4188,29 +4188,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse)other);
         } else {
@@ -4234,13 +4234,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4284,12 +4284,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4307,22 +4307,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyColumnResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyColumnResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4334,7 +4334,7 @@ public final class MasterProtos {
 
   public interface MoveRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MoveRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -4366,29 +4366,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.MoveRegionRequest}
    */
   public  static final class MoveRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MoveRegionRequest)
       MoveRegionRequestOrBuilder {
     // Use MoveRegionRequest.newBuilder() to construct.
-    private MoveRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MoveRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MoveRegionRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MoveRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4432,22 +4432,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4521,7 +4521,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -4538,11 +4538,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getDestServerName());
       }
       size += unknownFields.getSerializedSize();
@@ -4597,61 +4597,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4669,7 +4669,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4677,15 +4677,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.MoveRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MoveRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4698,12 +4698,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
           getDestServerNameFieldBuilder();
@@ -4726,7 +4726,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionRequest_descriptor;
       }
@@ -4772,29 +4772,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest)other);
         } else {
@@ -4832,13 +4832,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4851,7 +4851,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -4954,11 +4954,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -4969,7 +4969,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName destServerName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> destServerNameBuilder_;
       /**
        * optional .hbase.pb.ServerName dest_server_name = 2;
@@ -5072,11 +5072,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.ServerName dest_server_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getDestServerNameFieldBuilder() {
         if (destServerNameBuilder_ == null) {
-          destServerNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          destServerNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getDestServerName(),
                   getParentForChildren(),
@@ -5086,12 +5086,12 @@ public final class MasterProtos {
         return destServerNameBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5109,22 +5109,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MoveRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MoveRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5136,34 +5136,34 @@ public final class MasterProtos {
 
   public interface MoveRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MoveRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.MoveRegionResponse}
    */
   public  static final class MoveRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MoveRegionResponse)
       MoveRegionResponseOrBuilder {
     // Use MoveRegionResponse.newBuilder() to construct.
-    private MoveRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MoveRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MoveRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MoveRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5181,22 +5181,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5213,7 +5213,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -5257,61 +5257,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5329,7 +5329,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5337,15 +5337,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.MoveRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MoveRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5358,12 +5358,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -5372,7 +5372,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MoveRegionResponse_descriptor;
       }
@@ -5399,29 +5399,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse)other);
         } else {
@@ -5442,13 +5442,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5459,12 +5459,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5482,22 +5482,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MoveRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MoveRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5509,7 +5509,7 @@ public final class MasterProtos {
 
   public interface DispatchMergingRegionsRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region_a = 1;
@@ -5573,11 +5573,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
    */
   public  static final class DispatchMergingRegionsRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DispatchMergingRegionsRequest)
       DispatchMergingRegionsRequestOrBuilder {
     // Use DispatchMergingRegionsRequest.newBuilder() to construct.
-    private DispatchMergingRegionsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DispatchMergingRegionsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DispatchMergingRegionsRequest() {
@@ -5587,18 +5587,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DispatchMergingRegionsRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5657,22 +5657,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5793,7 +5793,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegionA());
@@ -5819,23 +5819,23 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegionA());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getRegionB());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, forcible_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonceGroup_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(5, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -5901,17 +5901,17 @@ public final class MasterProtos {
       }
       if (hasForcible()) {
         hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getForcible());
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -5920,61 +5920,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5992,7 +5992,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6005,15 +6005,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DispatchMergingRegionsRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6026,12 +6026,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionAFieldBuilder();
           getRegionBFieldBuilder();
@@ -6060,7 +6060,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
       }
@@ -6118,29 +6118,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)other);
         } else {
@@ -6188,13 +6188,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6207,7 +6207,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionA_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionABuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region_a = 1;
@@ -6310,11 +6310,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region_a = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionAFieldBuilder() {
         if (regionABuilder_ == null) {
-          regionABuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionABuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegionA(),
                   getParentForChildren(),
@@ -6325,7 +6325,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionB_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region_b = 2;
@@ -6428,11 +6428,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region_b = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionBFieldBuilder() {
         if (regionBBuilder_ == null) {
-          regionBBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegionB(),
                   getParentForChildren(),
@@ -6538,12 +6538,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6561,22 +6561,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DispatchMergingRegionsRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DispatchMergingRegionsRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6588,7 +6588,7 @@ public final class MasterProtos {
 
   public interface DispatchMergingRegionsResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -6603,11 +6603,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
    */
   public  static final class DispatchMergingRegionsResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DispatchMergingRegionsResponse)
       DispatchMergingRegionsResponseOrBuilder {
     // Use DispatchMergingRegionsResponse.newBuilder() to construct.
-    private DispatchMergingRegionsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DispatchMergingRegionsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DispatchMergingRegionsResponse() {
@@ -6615,18 +6615,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DispatchMergingRegionsResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6649,22 +6649,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6697,7 +6697,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -6711,7 +6711,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -6749,7 +6749,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -6758,61 +6758,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6830,7 +6830,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6838,15 +6838,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DispatchMergingRegionsResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6859,12 +6859,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -6875,7 +6875,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
       }
@@ -6909,29 +6909,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse)other);
         } else {
@@ -6955,13 +6955,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7005,12 +7005,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7028,22 +7028,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DispatchMergingRegionsResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DispatchMergingRegionsResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7055,7 +7055,7 @@ public final class MasterProtos {
 
   public interface AssignRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AssignRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -7074,29 +7074,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.AssignRegionRequest}
    */
   public  static final class AssignRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AssignRegionRequest)
       AssignRegionRequestOrBuilder {
     // Use AssignRegionRequest.newBuilder() to construct.
-    private AssignRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AssignRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AssignRegionRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AssignRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7127,22 +7127,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7189,7 +7189,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -7203,7 +7203,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       size += unknownFields.getSerializedSize();
@@ -7249,61 +7249,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7321,7 +7321,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7329,15 +7329,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.AssignRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AssignRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7350,12 +7350,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -7371,7 +7371,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionRequest_descriptor;
       }
@@ -7409,29 +7409,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest)other);
         } else {
@@ -7461,13 +7461,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7480,7 +7480,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -7583,11 +7583,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -7597,12 +7597,12 @@ public final class MasterProtos {
         return regionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7620,22 +7620,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AssignRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AssignRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7647,34 +7647,34 @@ public final class MasterProtos {
 
   public interface AssignRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AssignRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.AssignRegionResponse}
    */
   public  static final class AssignRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AssignRegionResponse)
       AssignRegionResponseOrBuilder {
     // Use AssignRegionResponse.newBuilder() to construct.
-    private AssignRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AssignRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AssignRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AssignRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7692,22 +7692,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7724,7 +7724,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -7768,61 +7768,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7840,7 +7840,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7848,15 +7848,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.AssignRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AssignRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7869,12 +7869,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7883,7 +7883,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AssignRegionResponse_descriptor;
       }
@@ -7910,29 +7910,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse)other);
         } else {
@@ -7953,13 +7953,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7970,12 +7970,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7993,22 +7993,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AssignRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AssignRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8020,7 +8020,7 @@ public final class MasterProtos {
 
   public interface UnassignRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UnassignRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -8048,11 +8048,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.UnassignRegionRequest}
    */
   public  static final class UnassignRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UnassignRegionRequest)
       UnassignRegionRequestOrBuilder {
     // Use UnassignRegionRequest.newBuilder() to construct.
-    private UnassignRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UnassignRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UnassignRegionRequest() {
@@ -8060,18 +8060,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UnassignRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8107,22 +8107,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8184,7 +8184,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -8201,11 +8201,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, force_);
       }
       size += unknownFields.getSerializedSize();
@@ -8252,7 +8252,7 @@ public final class MasterProtos {
       }
       if (hasForce()) {
         hash = (37 * hash) + FORCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getForce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -8261,61 +8261,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8333,7 +8333,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8341,15 +8341,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.UnassignRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UnassignRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8362,12 +8362,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -8385,7 +8385,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionRequest_descriptor;
       }
@@ -8427,29 +8427,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest)other);
         } else {
@@ -8482,13 +8482,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8501,7 +8501,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -8604,11 +8604,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -8650,12 +8650,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8673,22 +8673,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UnassignRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UnassignRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8700,34 +8700,34 @@ public final class MasterProtos {
 
   public interface UnassignRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UnassignRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.UnassignRegionResponse}
    */
   public  static final class UnassignRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UnassignRegionResponse)
       UnassignRegionResponseOrBuilder {
     // Use UnassignRegionResponse.newBuilder() to construct.
-    private UnassignRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UnassignRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UnassignRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UnassignRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8745,22 +8745,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8777,7 +8777,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -8821,61 +8821,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8893,7 +8893,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8901,15 +8901,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.UnassignRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UnassignRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8922,12 +8922,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -8936,7 +8936,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_UnassignRegionResponse_descriptor;
       }
@@ -8963,29 +8963,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse)other);
         } else {
@@ -9006,13 +9006,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9023,12 +9023,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9046,22 +9046,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UnassignRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UnassignRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9073,7 +9073,7 @@ public final class MasterProtos {
 
   public interface OfflineRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.OfflineRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -9092,29 +9092,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.OfflineRegionRequest}
    */
   public  static final class OfflineRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.OfflineRegionRequest)
       OfflineRegionRequestOrBuilder {
     // Use OfflineRegionRequest.newBuilder() to construct.
-    private OfflineRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private OfflineRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private OfflineRegionRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private OfflineRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9145,22 +9145,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9207,7 +9207,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -9221,7 +9221,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       size += unknownFields.getSerializedSize();
@@ -9267,61 +9267,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9339,7 +9339,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9347,15 +9347,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.OfflineRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.OfflineRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9368,12 +9368,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -9389,7 +9389,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionRequest_descriptor;
       }
@@ -9427,29 +9427,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest)other);
         } else {
@@ -9479,13 +9479,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9498,7 +9498,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -9601,11 +9601,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -9615,12 +9615,12 @@ public final class MasterProtos {
         return regionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9638,22 +9638,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public OfflineRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new OfflineRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9665,34 +9665,34 @@ public final class MasterProtos {
 
   public interface OfflineRegionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.OfflineRegionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.OfflineRegionResponse}
    */
   public  static final class OfflineRegionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.OfflineRegionResponse)
       OfflineRegionResponseOrBuilder {
     // Use OfflineRegionResponse.newBuilder() to construct.
-    private OfflineRegionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private OfflineRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private OfflineRegionResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private OfflineRegionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9710,22 +9710,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9742,7 +9742,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -9786,61 +9786,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9858,7 +9858,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9866,15 +9866,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.OfflineRegionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.OfflineRegionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9887,12 +9887,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -9901,7 +9901,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_OfflineRegionResponse_descriptor;
       }
@@ -9928,29 +9928,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse)other);
         } else {
@@ -9971,13 +9971,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9988,12 +9988,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10011,22 +10011,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public OfflineRegionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new OfflineRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10038,7 +10038,7 @@ public final class MasterProtos {
 
   public interface CreateTableRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableSchema table_schema = 1;
@@ -10056,7 +10056,7 @@ public final class MasterProtos {
     /**
      * repeated bytes split_keys = 2;
      */
-    java.util.List getSplitKeysList();
+    java.util.List getSplitKeysList();
     /**
      * repeated bytes split_keys = 2;
      */
@@ -10064,7 +10064,7 @@ public final class MasterProtos {
     /**
      * repeated bytes split_keys = 2;
      */
-    com.google.protobuf.ByteString getSplitKeys(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitKeys(int index);
 
     /**
      * optional uint64 nonce_group = 3 [default = 0];
@@ -10088,11 +10088,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.CreateTableRequest}
    */
   public  static final class CreateTableRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CreateTableRequest)
       CreateTableRequestOrBuilder {
     // Use CreateTableRequest.newBuilder() to construct.
-    private CreateTableRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CreateTableRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CreateTableRequest() {
@@ -10102,18 +10102,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CreateTableRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10144,7 +10144,7 @@ public final class MasterProtos {
             }
             case 18: {
               if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                splitKeys_ = new java.util.ArrayList();
+                splitKeys_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000002;
               }
               splitKeys_.add(input.readBytes());
@@ -10162,10 +10162,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -10175,12 +10175,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10210,11 +10210,11 @@ public final class MasterProtos {
     }
 
     public static final int SPLIT_KEYS_FIELD_NUMBER = 2;
-    private java.util.List splitKeys_;
+    private java.util.List splitKeys_;
     /**
      * repeated bytes split_keys = 2;
      */
-    public java.util.List
+    public java.util.List
         getSplitKeysList() {
       return splitKeys_;
     }
@@ -10227,7 +10227,7 @@ public final class MasterProtos {
     /**
      * repeated bytes split_keys = 2;
      */
-    public com.google.protobuf.ByteString getSplitKeys(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitKeys(int index) {
       return splitKeys_.get(index);
     }
 
@@ -10279,7 +10279,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableSchema());
@@ -10302,24 +10302,24 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableSchema());
       }
       {
         int dataSize = 0;
         for (int i = 0; i < splitKeys_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(splitKeys_.get(i));
         }
         size += dataSize;
         size += 1 * getSplitKeysList().size();
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -10377,12 +10377,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -10391,61 +10391,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10463,7 +10463,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10471,15 +10471,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.CreateTableRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CreateTableRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10492,12 +10492,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableSchemaFieldBuilder();
         }
@@ -10519,7 +10519,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableRequest_descriptor;
       }
@@ -10570,29 +10570,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest)other);
         } else {
@@ -10638,13 +10638,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -10657,7 +10657,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema table_schema = 1;
@@ -10760,11 +10760,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableSchema table_schema = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTableSchema(),
                   getParentForChildren(),
@@ -10774,17 +10774,17 @@ public final class MasterProtos {
         return tableSchemaBuilder_;
       }
 
-      private java.util.List splitKeys_ = java.util.Collections.emptyList();
+      private java.util.List splitKeys_ = java.util.Collections.emptyList();
       private void ensureSplitKeysIsMutable() {
         if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          splitKeys_ = new java.util.ArrayList(splitKeys_);
+          splitKeys_ = new java.util.ArrayList(splitKeys_);
           bitField0_ |= 0x00000002;
          }
       }
       /**
        * repeated bytes split_keys = 2;
        */
-      public java.util.List
+      public java.util.List
           getSplitKeysList() {
         return java.util.Collections.unmodifiableList(splitKeys_);
       }
@@ -10797,14 +10797,14 @@ public final class MasterProtos {
       /**
        * repeated bytes split_keys = 2;
        */
-      public com.google.protobuf.ByteString getSplitKeys(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitKeys(int index) {
         return splitKeys_.get(index);
       }
       /**
        * repeated bytes split_keys = 2;
        */
       public Builder setSplitKeys(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10816,7 +10816,7 @@ public final class MasterProtos {
       /**
        * repeated bytes split_keys = 2;
        */
-      public Builder addSplitKeys(com.google.protobuf.ByteString value) {
+      public Builder addSplitKeys(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -10829,9 +10829,9 @@ public final class MasterProtos {
        * repeated bytes split_keys = 2;
        */
       public Builder addAllSplitKeys(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureSplitKeysIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, splitKeys_);
         onChanged();
         return this;
@@ -10910,12 +10910,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10933,22 +10933,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CreateTableRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CreateTableRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10960,7 +10960,7 @@ public final class MasterProtos {
 
   public interface CreateTableResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -10975,11 +10975,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.CreateTableResponse}
    */
   public  static final class CreateTableResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CreateTableResponse)
       CreateTableResponseOrBuilder {
     // Use CreateTableResponse.newBuilder() to construct.
-    private CreateTableResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CreateTableResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CreateTableResponse() {
@@ -10987,18 +10987,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CreateTableResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11021,22 +11021,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11069,7 +11069,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -11083,7 +11083,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -11121,7 +11121,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -11130,61 +11130,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11202,7 +11202,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11210,15 +11210,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.CreateTableResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CreateTableResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11231,12 +11231,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -11247,7 +11247,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateTableResponse_descriptor;
       }
@@ -11281,29 +11281,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse)other);
         } else {
@@ -11327,13 +11327,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11377,12 +11377,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11400,22 +11400,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CreateTableResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CreateTableResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11427,7 +11427,7 @@ public final class MasterProtos {
 
   public interface DeleteTableRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteTableRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -11464,11 +11464,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteTableRequest}
    */
   public  static final class DeleteTableRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteTableRequest)
       DeleteTableRequestOrBuilder {
     // Use DeleteTableRequest.newBuilder() to construct.
-    private DeleteTableRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteTableRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteTableRequest() {
@@ -11477,18 +11477,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteTableRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11529,22 +11529,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11621,7 +11621,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -11641,15 +11641,15 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -11701,12 +11701,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -11715,61 +11715,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11787,7 +11787,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11795,15 +11795,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteTableRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteTableRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11816,12 +11816,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -11841,7 +11841,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableRequest_descriptor;
       }
@@ -11887,29 +11887,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest)other);
         } else {
@@ -11945,13 +11945,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11964,7 +11964,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -12067,11 +12067,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -12145,12 +12145,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12168,22 +12168,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteTableRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteTableRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12195,7 +12195,7 @@ public final class MasterProtos {
 
   public interface DeleteTableResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteTableResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -12210,11 +12210,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteTableResponse}
    */
   public  static final class DeleteTableResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteTableResponse)
       DeleteTableResponseOrBuilder {
     // Use DeleteTableResponse.newBuilder() to construct.
-    private DeleteTableResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteTableResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteTableResponse() {
@@ -12222,18 +12222,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteTableResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12256,22 +12256,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12304,7 +12304,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -12318,7 +12318,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -12356,7 +12356,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -12365,61 +12365,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -12437,7 +12437,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -12445,15 +12445,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteTableResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteTableResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -12466,12 +12466,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -12482,7 +12482,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteTableResponse_descriptor;
       }
@@ -12516,29 +12516,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse)other);
         } else {
@@ -12562,13 +12562,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12612,12 +12612,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12635,22 +12635,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteTableResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteTableResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12662,7 +12662,7 @@ public final class MasterProtos {
 
   public interface TruncateTableRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TruncateTableRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName tableName = 1;
@@ -12708,11 +12708,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.TruncateTableRequest}
    */
   public  static final class TruncateTableRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TruncateTableRequest)
       TruncateTableRequestOrBuilder {
     // Use TruncateTableRequest.newBuilder() to construct.
-    private TruncateTableRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TruncateTableRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TruncateTableRequest() {
@@ -12722,18 +12722,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TruncateTableRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -12779,22 +12779,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -12886,7 +12886,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -12909,19 +12909,19 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, preserveSplits_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonceGroup_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -12978,17 +12978,17 @@ public final class MasterProtos {
       }
       if (hasPreserveSplits()) {
         hash = (37 * hash) + PRESERVESPLITS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getPreserveSplits());
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -12997,61 +12997,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -13069,7 +13069,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -13077,15 +13077,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.TruncateTableRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TruncateTableRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13098,12 +13098,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -13125,7 +13125,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableRequest_descriptor;
       }
@@ -13175,29 +13175,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest)other);
         } else {
@@ -13236,13 +13236,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13255,7 +13255,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName tableName = 1;
@@ -13358,11 +13358,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName tableName = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -13468,12 +13468,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -13491,22 +13491,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TruncateTableRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TruncateTableRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13518,7 +13518,7 @@ public final class MasterProtos {
 
   public interface TruncateTableResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TruncateTableResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -13533,11 +13533,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.TruncateTableResponse}
    */
   public  static final class TruncateTableResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TruncateTableResponse)
       TruncateTableResponseOrBuilder {
     // Use TruncateTableResponse.newBuilder() to construct.
-    private TruncateTableResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TruncateTableResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TruncateTableResponse() {
@@ -13545,18 +13545,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TruncateTableResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -13579,22 +13579,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -13627,7 +13627,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -13641,7 +13641,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -13679,7 +13679,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -13688,61 +13688,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -13760,7 +13760,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -13768,15 +13768,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.TruncateTableResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TruncateTableResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -13789,12 +13789,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -13805,7 +13805,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_TruncateTableResponse_descriptor;
       }
@@ -13839,29 +13839,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse)other);
         } else {
@@ -13885,13 +13885,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -13935,12 +13935,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -13958,22 +13958,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TruncateTableResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TruncateTableResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -13985,7 +13985,7 @@ public final class MasterProtos {
 
   public interface EnableTableRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.EnableTableRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -14022,11 +14022,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.EnableTableRequest}
    */
   public  static final class EnableTableRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.EnableTableRequest)
       EnableTableRequestOrBuilder {
     // Use EnableTableRequest.newBuilder() to construct.
-    private EnableTableRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private EnableTableRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private EnableTableRequest() {
@@ -14035,18 +14035,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private EnableTableRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14087,22 +14087,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14179,7 +14179,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -14199,15 +14199,15 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -14259,12 +14259,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -14273,61 +14273,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14345,7 +14345,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -14353,15 +14353,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.EnableTableRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.EnableTableRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -14374,12 +14374,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -14399,7 +14399,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableRequest_descriptor;
       }
@@ -14445,29 +14445,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest)other);
         } else {
@@ -14503,13 +14503,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -14522,7 +14522,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -14625,11 +14625,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -14703,12 +14703,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -14726,22 +14726,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public EnableTableRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new EnableTableRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -14753,7 +14753,7 @@ public final class MasterProtos {
 
   public interface EnableTableResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.EnableTableResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -14768,11 +14768,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.EnableTableResponse}
    */
   public  static final class EnableTableResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.EnableTableResponse)
       EnableTableResponseOrBuilder {
     // Use EnableTableResponse.newBuilder() to construct.
-    private EnableTableResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private EnableTableResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private EnableTableResponse() {
@@ -14780,18 +14780,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private EnableTableResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -14814,22 +14814,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -14862,7 +14862,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -14876,7 +14876,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -14914,7 +14914,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -14923,61 +14923,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -14995,7 +14995,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15003,15 +15003,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.EnableTableResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.EnableTableResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15024,12 +15024,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -15040,7 +15040,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableTableResponse_descriptor;
       }
@@ -15074,29 +15074,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse)other);
         } else {
@@ -15120,13 +15120,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15170,12 +15170,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15193,22 +15193,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public EnableTableResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new EnableTableResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15220,7 +15220,7 @@ public final class MasterProtos {
 
   public interface DisableTableRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DisableTableRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -15257,11 +15257,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DisableTableRequest}
    */
   public  static final class DisableTableRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DisableTableRequest)
       DisableTableRequestOrBuilder {
     // Use DisableTableRequest.newBuilder() to construct.
-    private DisableTableRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DisableTableRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DisableTableRequest() {
@@ -15270,18 +15270,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DisableTableRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -15322,22 +15322,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -15414,7 +15414,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -15434,15 +15434,15 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -15494,12 +15494,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -15508,61 +15508,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -15580,7 +15580,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -15588,15 +15588,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DisableTableRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DisableTableRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -15609,12 +15609,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -15634,7 +15634,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableRequest_descriptor;
       }
@@ -15680,29 +15680,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest)other);
         } else {
@@ -15738,13 +15738,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -15757,7 +15757,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -15860,11 +15860,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -15938,12 +15938,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -15961,22 +15961,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DisableTableRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DisableTableRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -15988,7 +15988,7 @@ public final class MasterProtos {
 
   public interface DisableTableResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DisableTableResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -16003,11 +16003,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DisableTableResponse}
    */
   public  static final class DisableTableResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DisableTableResponse)
       DisableTableResponseOrBuilder {
     // Use DisableTableResponse.newBuilder() to construct.
-    private DisableTableResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DisableTableResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DisableTableResponse() {
@@ -16015,18 +16015,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DisableTableResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16049,22 +16049,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16097,7 +16097,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -16111,7 +16111,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -16149,7 +16149,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -16158,61 +16158,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16230,7 +16230,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16238,15 +16238,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DisableTableResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DisableTableResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16259,12 +16259,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -16275,7 +16275,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DisableTableResponse_descriptor;
       }
@@ -16309,29 +16309,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse)other);
         } else {
@@ -16355,13 +16355,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -16405,12 +16405,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -16428,22 +16428,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DisableTableResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DisableTableResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -16455,7 +16455,7 @@ public final class MasterProtos {
 
   public interface ModifyTableRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyTableRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -16505,11 +16505,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ModifyTableRequest}
    */
   public  static final class ModifyTableRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyTableRequest)
       ModifyTableRequestOrBuilder {
     // Use ModifyTableRequest.newBuilder() to construct.
-    private ModifyTableRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyTableRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyTableRequest() {
@@ -16518,18 +16518,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyTableRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -16583,22 +16583,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -16704,7 +16704,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -16727,19 +16727,19 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTableSchema());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonceGroup_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -16800,12 +16800,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -16814,61 +16814,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -16886,7 +16886,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -16894,15 +16894,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ModifyTableRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyTableRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -16915,12 +16915,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getTableSchemaFieldBuilder();
@@ -16947,7 +16947,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableRequest_descriptor;
       }
@@ -17001,29 +17001,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest)other);
         } else {
@@ -17068,13 +17068,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -17087,7 +17087,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -17190,11 +17190,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -17205,7 +17205,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema table_schema = 2;
@@ -17308,11 +17308,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableSchema table_schema = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTableSchema(),
                   getParentForChildren(),
@@ -17386,12 +17386,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17409,22 +17409,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyTableRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyTableRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17436,7 +17436,7 @@ public final class MasterProtos {
 
   public interface ModifyTableResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyTableResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -17451,11 +17451,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ModifyTableResponse}
    */
   public  static final class ModifyTableResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyTableResponse)
       ModifyTableResponseOrBuilder {
     // Use ModifyTableResponse.newBuilder() to construct.
-    private ModifyTableResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyTableResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyTableResponse() {
@@ -17463,18 +17463,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyTableResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -17497,22 +17497,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -17545,7 +17545,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -17559,7 +17559,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -17597,7 +17597,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -17606,61 +17606,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -17678,7 +17678,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -17686,15 +17686,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ModifyTableResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyTableResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -17707,12 +17707,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -17723,7 +17723,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyTableResponse_descriptor;
       }
@@ -17757,29 +17757,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse)other);
         } else {
@@ -17803,13 +17803,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -17853,12 +17853,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -17876,22 +17876,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyTableResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyTableResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -17903,7 +17903,7 @@ public final class MasterProtos {
 
   public interface CreateNamespaceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateNamespaceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -17940,11 +17940,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.CreateNamespaceRequest}
    */
   public  static final class CreateNamespaceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CreateNamespaceRequest)
       CreateNamespaceRequestOrBuilder {
     // Use CreateNamespaceRequest.newBuilder() to construct.
-    private CreateNamespaceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CreateNamespaceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CreateNamespaceRequest() {
@@ -17953,18 +17953,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CreateNamespaceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -18005,22 +18005,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -18097,7 +18097,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getNamespaceDescriptor());
@@ -18117,15 +18117,15 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getNamespaceDescriptor());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -18177,12 +18177,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -18191,61 +18191,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -18263,7 +18263,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -18271,15 +18271,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.CreateNamespaceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CreateNamespaceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -18292,12 +18292,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
         }
@@ -18317,7 +18317,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceRequest_descriptor;
       }
@@ -18363,29 +18363,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest)other);
         } else {
@@ -18421,13 +18421,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -18440,7 +18440,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
       /**
        * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -18543,11 +18543,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getNamespaceDescriptor(),
                   getParentForChildren(),
@@ -18621,12 +18621,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -18644,22 +18644,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CreateNamespaceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CreateNamespaceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -18671,7 +18671,7 @@ public final class MasterProtos {
 
   public interface CreateNamespaceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateNamespaceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -18686,11 +18686,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.CreateNamespaceResponse}
    */
   public  static final class CreateNamespaceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CreateNamespaceResponse)
       CreateNamespaceResponseOrBuilder {
     // Use CreateNamespaceResponse.newBuilder() to construct.
-    private CreateNamespaceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CreateNamespaceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CreateNamespaceResponse() {
@@ -18698,18 +18698,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CreateNamespaceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -18732,22 +18732,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -18780,7 +18780,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -18794,7 +18794,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -18832,7 +18832,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -18841,61 +18841,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -18913,7 +18913,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -18921,15 +18921,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.CreateNamespaceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CreateNamespaceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -18942,12 +18942,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -18958,7 +18958,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CreateNamespaceResponse_descriptor;
       }
@@ -18992,29 +18992,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse)other);
         } else {
@@ -19038,13 +19038,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -19088,12 +19088,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -19111,22 +19111,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CreateNamespaceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CreateNamespaceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -19138,7 +19138,7 @@ public final class MasterProtos {
 
   public interface DeleteNamespaceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteNamespaceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string namespaceName = 1;
@@ -19151,7 +19151,7 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes();
 
     /**
@@ -19176,11 +19176,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteNamespaceRequest}
    */
   public  static final class DeleteNamespaceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteNamespaceRequest)
       DeleteNamespaceRequestOrBuilder {
     // Use DeleteNamespaceRequest.newBuilder() to construct.
-    private DeleteNamespaceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteNamespaceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteNamespaceRequest() {
@@ -19190,18 +19190,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteNamespaceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19218,7 +19218,7 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               namespaceName_ = bs;
               break;
@@ -19235,22 +19235,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19274,8 +19274,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespaceName_ = s;
@@ -19286,17 +19286,17 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes() {
       java.lang.Object ref = namespaceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespaceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -19344,10 +19344,10 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt64(2, nonceGroup_);
@@ -19364,14 +19364,14 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -19423,12 +19423,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -19437,61 +19437,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -19509,7 +19509,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -19517,15 +19517,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteNamespaceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteNamespaceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -19538,12 +19538,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -19558,7 +19558,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceRequest_descriptor;
       }
@@ -19600,29 +19600,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest)other);
         } else {
@@ -19657,13 +19657,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -19688,8 +19688,8 @@ public final class MasterProtos {
       public java.lang.String getNamespaceName() {
         java.lang.Object ref = namespaceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespaceName_ = s;
@@ -19702,17 +19702,17 @@ public final class MasterProtos {
       /**
        * required string namespaceName = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceNameBytes() {
         java.lang.Object ref = namespaceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespaceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -19741,7 +19741,7 @@ public final class MasterProtos {
        * required string namespaceName = 1;
        */
       public Builder setNamespaceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -19815,12 +19815,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -19838,22 +19838,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteNamespaceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteNamespaceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -19865,7 +19865,7 @@ public final class MasterProtos {
 
   public interface DeleteNamespaceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteNamespaceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -19880,11 +19880,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteNamespaceResponse}
    */
   public  static final class DeleteNamespaceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteNamespaceResponse)
       DeleteNamespaceResponseOrBuilder {
     // Use DeleteNamespaceResponse.newBuilder() to construct.
-    private DeleteNamespaceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteNamespaceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteNamespaceResponse() {
@@ -19892,18 +19892,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteNamespaceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -19926,22 +19926,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -19974,7 +19974,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -19988,7 +19988,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -20026,7 +20026,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -20035,61 +20035,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -20107,7 +20107,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -20115,15 +20115,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteNamespaceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteNamespaceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -20136,12 +20136,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -20152,7 +20152,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteNamespaceResponse_descriptor;
       }
@@ -20186,29 +20186,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse)other);
         } else {
@@ -20232,13 +20232,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20282,12 +20282,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -20305,22 +20305,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteNamespaceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteNamespaceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -20332,7 +20332,7 @@ public final class MasterProtos {
 
   public interface ModifyNamespaceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyNamespaceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -20369,11 +20369,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ModifyNamespaceRequest}
    */
   public  static final class ModifyNamespaceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyNamespaceRequest)
       ModifyNamespaceRequestOrBuilder {
     // Use ModifyNamespaceRequest.newBuilder() to construct.
-    private ModifyNamespaceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyNamespaceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyNamespaceRequest() {
@@ -20382,18 +20382,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyNamespaceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -20434,22 +20434,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -20526,7 +20526,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getNamespaceDescriptor());
@@ -20546,15 +20546,15 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getNamespaceDescriptor());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -20606,12 +20606,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -20620,61 +20620,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -20692,7 +20692,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -20700,15 +20700,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ModifyNamespaceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyNamespaceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -20721,12 +20721,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
         }
@@ -20746,7 +20746,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceRequest_descriptor;
       }
@@ -20792,29 +20792,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest)other);
         } else {
@@ -20850,13 +20850,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -20869,7 +20869,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
       /**
        * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -20972,11 +20972,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getNamespaceDescriptor(),
                   getParentForChildren(),
@@ -21050,12 +21050,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -21073,22 +21073,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyNamespaceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyNamespaceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -21100,7 +21100,7 @@ public final class MasterProtos {
 
   public interface ModifyNamespaceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ModifyNamespaceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint64 proc_id = 1;
@@ -21115,11 +21115,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ModifyNamespaceResponse}
    */
   public  static final class ModifyNamespaceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ModifyNamespaceResponse)
       ModifyNamespaceResponseOrBuilder {
     // Use ModifyNamespaceResponse.newBuilder() to construct.
-    private ModifyNamespaceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ModifyNamespaceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ModifyNamespaceResponse() {
@@ -21127,18 +21127,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ModifyNamespaceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -21161,22 +21161,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -21209,7 +21209,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -21223,7 +21223,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -21261,7 +21261,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -21270,61 +21270,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -21342,7 +21342,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -21350,15 +21350,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ModifyNamespaceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ModifyNamespaceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21371,12 +21371,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -21387,7 +21387,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ModifyNamespaceResponse_descriptor;
       }
@@ -21421,29 +21421,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse)other);
         } else {
@@ -21467,13 +21467,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -21517,12 +21517,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -21540,22 +21540,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ModifyNamespaceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ModifyNamespaceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -21567,7 +21567,7 @@ public final class MasterProtos {
 
   public interface GetNamespaceDescriptorRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetNamespaceDescriptorRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string namespaceName = 1;
@@ -21580,18 +21580,18 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.GetNamespaceDescriptorRequest}
    */
   public  static final class GetNamespaceDescriptorRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetNamespaceDescriptorRequest)
       GetNamespaceDescriptorRequestOrBuilder {
     // Use GetNamespaceDescriptorRequest.newBuilder() to construct.
-    private GetNamespaceDescriptorRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetNamespaceDescriptorRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetNamespaceDescriptorRequest() {
@@ -21599,18 +21599,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetNamespaceDescriptorRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -21627,29 +21627,29 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               namespaceName_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -21673,8 +21673,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespaceName_ = s;
@@ -21685,17 +21685,17 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes() {
       java.lang.Object ref = namespaceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespaceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -21713,10 +21713,10 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
       }
       unknownFields.writeTo(output);
     }
@@ -21727,7 +21727,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -21772,61 +21772,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -21844,7 +21844,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -21852,15 +21852,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetNamespaceDescriptorRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetNamespaceDescriptorRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -21873,12 +21873,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -21889,7 +21889,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor;
       }
@@ -21923,29 +21923,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest)other);
         } else {
@@ -21974,13 +21974,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -22005,8 +22005,8 @@ public final class MasterProtos {
       public java.lang.String getNamespaceName() {
         java.lang.Object ref = namespaceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespaceName_ = s;
@@ -22019,17 +22019,17 @@ public final class MasterProtos {
       /**
        * required string namespaceName = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceNameBytes() {
         java.lang.Object ref = namespaceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespaceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -22058,7 +22058,7 @@ public final class MasterProtos {
        * required string namespaceName = 1;
        */
       public Builder setNamespaceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -22068,12 +22068,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -22091,22 +22091,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetNamespaceDescriptorRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetNamespaceDescriptorRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -22118,7 +22118,7 @@ public final class MasterProtos {
 
   public interface GetNamespaceDescriptorResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetNamespaceDescriptorResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -22137,29 +22137,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetNamespaceDescriptorResponse}
    */
   public  static final class GetNamespaceDescriptorResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetNamespaceDescriptorResponse)
       GetNamespaceDescriptorResponseOrBuilder {
     // Use GetNamespaceDescriptorResponse.newBuilder() to construct.
-    private GetNamespaceDescriptorResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetNamespaceDescriptorResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetNamespaceDescriptorResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetNamespaceDescriptorResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -22190,22 +22190,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -22252,7 +22252,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getNamespaceDescriptor());
@@ -22266,7 +22266,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getNamespaceDescriptor());
       }
       size += unknownFields.getSerializedSize();
@@ -22312,61 +22312,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -22384,7 +22384,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -22392,15 +22392,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetNamespaceDescriptorResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetNamespaceDescriptorResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -22413,12 +22413,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
         }
@@ -22434,7 +22434,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetNamespaceDescriptorResponse_descriptor;
       }
@@ -22472,29 +22472,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse)other);
         } else {
@@ -22524,13 +22524,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -22543,7 +22543,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor namespaceDescriptor_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
       /**
        * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -22646,11 +22646,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   getNamespaceDescriptor(),
                   getParentForChildren(),
@@ -22660,12 +22660,12 @@ public final class MasterProtos {
         return namespaceDescriptorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -22683,22 +22683,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetNamespaceDescriptorResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetNamespaceDescriptorResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -22710,34 +22710,34 @@ public final class MasterProtos {
 
   public interface ListNamespaceDescriptorsRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListNamespaceDescriptorsRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.ListNamespaceDescriptorsRequest}
    */
   public  static final class ListNamespaceDescriptorsRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListNamespaceDescriptorsRequest)
       ListNamespaceDescriptorsRequestOrBuilder {
     // Use ListNamespaceDescriptorsRequest.newBuilder() to construct.
-    private ListNamespaceDescriptorsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListNamespaceDescriptorsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListNamespaceDescriptorsRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListNamespaceDescriptorsRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -22755,22 +22755,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -22787,7 +22787,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -22831,61 +22831,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -22903,7 +22903,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -22911,15 +22911,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListNamespaceDescriptorsRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListNamespaceDescriptorsRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -22932,12 +22932,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -22946,7 +22946,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsRequest_descriptor;
       }
@@ -22973,29 +22973,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest)other);
         } else {
@@ -23016,13 +23016,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -23033,12 +23033,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -23056,22 +23056,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListNamespaceDescriptorsRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListNamespaceDescriptorsRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -23083,7 +23083,7 @@ public final class MasterProtos {
 
   public interface ListNamespaceDescriptorsResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListNamespaceDescriptorsResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.NamespaceDescriptor namespaceDescriptor = 1;
@@ -23113,11 +23113,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ListNamespaceDescriptorsResponse}
    */
   public  static final class ListNamespaceDescriptorsResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListNamespaceDescriptorsResponse)
       ListNamespaceDescriptorsResponseOrBuilder {
     // Use ListNamespaceDescriptorsResponse.newBuilder() to construct.
-    private ListNamespaceDescriptorsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListNamespaceDescriptorsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListNamespaceDescriptorsResponse() {
@@ -23125,18 +23125,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListNamespaceDescriptorsResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -23163,10 +23163,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -23176,12 +23176,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -23239,7 +23239,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < namespaceDescriptor_.size(); i++) {
         output.writeMessage(1, namespaceDescriptor_.get(i));
@@ -23253,7 +23253,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < namespaceDescriptor_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, namespaceDescriptor_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -23296,61 +23296,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -23368,7 +23368,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -23376,15 +23376,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListNamespaceDescriptorsResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListNamespaceDescriptorsResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -23397,12 +23397,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNamespaceDescriptorFieldBuilder();
         }
@@ -23418,7 +23418,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListNamespaceDescriptorsResponse_descriptor;
       }
@@ -23455,29 +23455,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse)other);
         } else {
@@ -23507,7 +23507,7 @@ public final class MasterProtos {
               namespaceDescriptor_ = other.namespaceDescriptor_;
               bitField0_ = (bitField0_ & ~0x00000001);
               namespaceDescriptorBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getNamespaceDescriptorFieldBuilder() : null;
             } else {
               namespaceDescriptorBuilder_.addAllMessages(other.namespaceDescriptor_);
@@ -23529,13 +23529,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -23556,7 +23556,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> namespaceDescriptorBuilder_;
 
       /**
@@ -23688,7 +23688,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (namespaceDescriptorBuilder_ == null) {
           ensureNamespaceDescriptorIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, namespaceDescriptor_);
           onChanged();
         } else {
@@ -23772,11 +23772,11 @@ public final class MasterProtos {
            getNamespaceDescriptorBuilderList() {
         return getNamespaceDescriptorFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder> 
           getNamespaceDescriptorFieldBuilder() {
         if (namespaceDescriptorBuilder_ == null) {
-          namespaceDescriptorBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          namespaceDescriptorBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NamespaceDescriptorOrBuilder>(
                   namespaceDescriptor_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -23787,12 +23787,12 @@ public final class MasterProtos {
         return namespaceDescriptorBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -23810,22 +23810,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListNamespaceDescriptorsResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListNamespaceDescriptorsResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -23837,7 +23837,7 @@ public final class MasterProtos {
 
   public interface ListTableDescriptorsByNamespaceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListTableDescriptorsByNamespaceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string namespaceName = 1;
@@ -23850,18 +23850,18 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.ListTableDescriptorsByNamespaceRequest}
    */
   public  static final class ListTableDescriptorsByNamespaceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListTableDescriptorsByNamespaceRequest)
       ListTableDescriptorsByNamespaceRequestOrBuilder {
     // Use ListTableDescriptorsByNamespaceRequest.newBuilder() to construct.
-    private ListTableDescriptorsByNamespaceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListTableDescriptorsByNamespaceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListTableDescriptorsByNamespaceRequest() {
@@ -23869,18 +23869,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListTableDescriptorsByNamespaceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -23897,29 +23897,29 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               namespaceName_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -23943,8 +23943,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespaceName_ = s;
@@ -23955,17 +23955,17 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes() {
       java.lang.Object ref = namespaceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespaceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -23983,10 +23983,10 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
       }
       unknownFields.writeTo(output);
     }
@@ -23997,7 +23997,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -24042,61 +24042,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -24114,7 +24114,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -24122,15 +24122,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListTableDescriptorsByNamespaceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListTableDescriptorsByNamespaceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -24143,12 +24143,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -24159,7 +24159,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_descriptor;
       }
@@ -24193,29 +24193,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)other);
         } else {
@@ -24244,13 +24244,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -24275,8 +24275,8 @@ public final class MasterProtos {
       public java.lang.String getNamespaceName() {
         java.lang.Object ref = namespaceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespaceName_ = s;
@@ -24289,17 +24289,17 @@ public final class MasterProtos {
       /**
        * required string namespaceName = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceNameBytes() {
         java.lang.Object ref = namespaceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespaceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -24328,7 +24328,7 @@ public final class MasterProtos {
        * required string namespaceName = 1;
        */
       public Builder setNamespaceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -24338,12 +24338,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -24361,22 +24361,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListTableDescriptorsByNamespaceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListTableDescriptorsByNamespaceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -24388,7 +24388,7 @@ public final class MasterProtos {
 
   public interface ListTableDescriptorsByNamespaceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListTableDescriptorsByNamespaceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.TableSchema tableSchema = 1;
@@ -24418,11 +24418,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ListTableDescriptorsByNamespaceResponse}
    */
   public  static final class ListTableDescriptorsByNamespaceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListTableDescriptorsByNamespaceResponse)
       ListTableDescriptorsByNamespaceResponseOrBuilder {
     // Use ListTableDescriptorsByNamespaceResponse.newBuilder() to construct.
-    private ListTableDescriptorsByNamespaceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListTableDescriptorsByNamespaceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListTableDescriptorsByNamespaceResponse() {
@@ -24430,18 +24430,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListTableDescriptorsByNamespaceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -24468,10 +24468,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -24481,12 +24481,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -24544,7 +24544,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < tableSchema_.size(); i++) {
         output.writeMessage(1, tableSchema_.get(i));
@@ -24558,7 +24558,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < tableSchema_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, tableSchema_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -24601,61 +24601,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -24673,7 +24673,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -24681,15 +24681,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListTableDescriptorsByNamespaceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListTableDescriptorsByNamespaceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -24702,12 +24702,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableSchemaFieldBuilder();
         }
@@ -24723,7 +24723,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_descriptor;
       }
@@ -24760,29 +24760,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse)other);
         } else {
@@ -24812,7 +24812,7 @@ public final class MasterProtos {
               tableSchema_ = other.tableSchema_;
               bitField0_ = (bitField0_ & ~0x00000001);
               tableSchemaBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableSchemaFieldBuilder() : null;
             } else {
               tableSchemaBuilder_.addAllMessages(other.tableSchema_);
@@ -24834,13 +24834,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -24861,7 +24861,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
 
       /**
@@ -24993,7 +24993,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (tableSchemaBuilder_ == null) {
           ensureTableSchemaIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, tableSchema_);
           onChanged();
         } else {
@@ -25077,11 +25077,11 @@ public final class MasterProtos {
            getTableSchemaBuilderList() {
         return getTableSchemaFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   tableSchema_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -25092,12 +25092,12 @@ public final class MasterProtos {
         return tableSchemaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -25115,22 +25115,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListTableDescriptorsByNamespaceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListTableDescriptorsByNamespaceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -25142,7 +25142,7 @@ public final class MasterProtos {
 
   public interface ListTableNamesByNamespaceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListTableNamesByNamespaceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string namespaceName = 1;
@@ -25155,18 +25155,18 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.ListTableNamesByNamespaceRequest}
    */
   public  static final class ListTableNamesByNamespaceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListTableNamesByNamespaceRequest)
       ListTableNamesByNamespaceRequestOrBuilder {
     // Use ListTableNamesByNamespaceRequest.newBuilder() to construct.
-    private ListTableNamesByNamespaceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListTableNamesByNamespaceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListTableNamesByNamespaceRequest() {
@@ -25174,18 +25174,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListTableNamesByNamespaceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -25202,29 +25202,29 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               namespaceName_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -25248,8 +25248,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespaceName_ = s;
@@ -25260,17 +25260,17 @@ public final class MasterProtos {
     /**
      * required string namespaceName = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceNameBytes() {
       java.lang.Object ref = namespaceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespaceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -25288,10 +25288,10 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, namespaceName_);
       }
       unknownFields.writeTo(output);
     }
@@ -25302,7 +25302,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, namespaceName_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -25347,61 +25347,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -25419,7 +25419,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -25427,15 +25427,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListTableNamesByNamespaceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListTableNamesByNamespaceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -25448,12 +25448,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -25464,7 +25464,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceRequest_descriptor;
       }
@@ -25498,29 +25498,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)other);
         } else {
@@ -25549,13 +25549,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -25580,8 +25580,8 @@ public final class MasterProtos {
       public java.lang.String getNamespaceName() {
         java.lang.Object ref = namespaceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespaceName_ = s;
@@ -25594,17 +25594,17 @@ public final class MasterProtos {
       /**
        * required string namespaceName = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceNameBytes() {
         java.lang.Object ref = namespaceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespaceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -25633,7 +25633,7 @@ public final class MasterProtos {
        * required string namespaceName = 1;
        */
       public Builder setNamespaceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -25643,12 +25643,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -25666,22 +25666,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListTableNamesByNamespaceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListTableNamesByNamespaceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -25693,7 +25693,7 @@ public final class MasterProtos {
 
   public interface ListTableNamesByNamespaceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListTableNamesByNamespaceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.TableName tableName = 1;
@@ -25723,11 +25723,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ListTableNamesByNamespaceResponse}
    */
   public  static final class ListTableNamesByNamespaceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListTableNamesByNamespaceResponse)
       ListTableNamesByNamespaceResponseOrBuilder {
     // Use ListTableNamesByNamespaceResponse.newBuilder() to construct.
-    private ListTableNamesByNamespaceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListTableNamesByNamespaceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListTableNamesByNamespaceResponse() {
@@ -25735,18 +25735,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListTableNamesByNamespaceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -25773,10 +25773,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -25786,12 +25786,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -25849,7 +25849,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < tableName_.size(); i++) {
         output.writeMessage(1, tableName_.get(i));
@@ -25863,7 +25863,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < tableName_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, tableName_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -25906,61 +25906,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -25978,7 +25978,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -25986,15 +25986,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListTableNamesByNamespaceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListTableNamesByNamespaceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -26007,12 +26007,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -26028,7 +26028,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTableNamesByNamespaceResponse_descriptor;
       }
@@ -26065,29 +26065,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse)other);
         } else {
@@ -26117,7 +26117,7 @@ public final class MasterProtos {
               tableName_ = other.tableName_;
               bitField0_ = (bitField0_ & ~0x00000001);
               tableNameBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableNameFieldBuilder() : null;
             } else {
               tableNameBuilder_.addAllMessages(other.tableName_);
@@ -26139,13 +26139,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -26166,7 +26166,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
 
       /**
@@ -26298,7 +26298,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (tableNameBuilder_ == null) {
           ensureTableNameIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, tableName_);
           onChanged();
         } else {
@@ -26382,11 +26382,11 @@ public final class MasterProtos {
            getTableNameBuilderList() {
         return getTableNameFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   tableName_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -26397,12 +26397,12 @@ public final class MasterProtos {
         return tableNameBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -26420,22 +26420,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListTableNamesByNamespaceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListTableNamesByNamespaceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -26447,34 +26447,34 @@ public final class MasterProtos {
 
   public interface ShutdownRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ShutdownRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.ShutdownRequest}
    */
   public  static final class ShutdownRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ShutdownRequest)
       ShutdownRequestOrBuilder {
     // Use ShutdownRequest.newBuilder() to construct.
-    private ShutdownRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ShutdownRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ShutdownRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ShutdownRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -26492,22 +26492,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -26524,7 +26524,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -26568,61 +26568,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -26640,7 +26640,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -26648,15 +26648,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ShutdownRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ShutdownRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -26669,12 +26669,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -26683,7 +26683,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownRequest_descriptor;
       }
@@ -26710,29 +26710,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest)other);
         } else {
@@ -26753,13 +26753,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -26770,12 +26770,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -26793,22 +26793,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ShutdownRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ShutdownRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -26820,34 +26820,34 @@ public final class MasterProtos {
 
   public interface ShutdownResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ShutdownResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.ShutdownResponse}
    */
   public  static final class ShutdownResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ShutdownResponse)
       ShutdownResponseOrBuilder {
     // Use ShutdownResponse.newBuilder() to construct.
-    private ShutdownResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ShutdownResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ShutdownResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ShutdownResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -26865,22 +26865,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -26897,7 +26897,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -26941,61 +26941,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -27013,7 +27013,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -27021,15 +27021,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ShutdownResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ShutdownResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -27042,12 +27042,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -27056,7 +27056,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ShutdownResponse_descriptor;
       }
@@ -27083,29 +27083,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse)other);
         } else {
@@ -27126,13 +27126,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -27143,12 +27143,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -27166,22 +27166,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ShutdownResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ShutdownResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -27193,34 +27193,34 @@ public final class MasterProtos {
 
   public interface StopMasterRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StopMasterRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.StopMasterRequest}
    */
   public  static final class StopMasterRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StopMasterRequest)
       StopMasterRequestOrBuilder {
     // Use StopMasterRequest.newBuilder() to construct.
-    private StopMasterRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StopMasterRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StopMasterRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StopMasterRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -27238,22 +27238,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -27270,7 +27270,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -27314,61 +27314,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -27386,7 +27386,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -27394,15 +27394,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.StopMasterRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StopMasterRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -27415,12 +27415,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -27429,7 +27429,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterRequest_descriptor;
       }
@@ -27456,29 +27456,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest)other);
         } else {
@@ -27499,13 +27499,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -27516,12 +27516,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -27539,22 +27539,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StopMasterRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StopMasterRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -27566,34 +27566,34 @@ public final class MasterProtos {
 
   public interface StopMasterResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StopMasterResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.StopMasterResponse}
    */
   public  static final class StopMasterResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StopMasterResponse)
       StopMasterResponseOrBuilder {
     // Use StopMasterResponse.newBuilder() to construct.
-    private StopMasterResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StopMasterResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StopMasterResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StopMasterResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -27611,22 +27611,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -27643,7 +27643,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -27687,61 +27687,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -27759,7 +27759,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -27767,15 +27767,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.StopMasterResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StopMasterResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -27788,12 +27788,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -27802,7 +27802,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_StopMasterResponse_descriptor;
       }
@@ -27829,29 +27829,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse)other);
         } else {
@@ -27872,13 +27872,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -27889,12 +27889,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -27912,22 +27912,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StopMasterResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StopMasterResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -27939,34 +27939,34 @@ public final class MasterProtos {
 
   public interface IsInMaintenanceModeRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsInMaintenanceModeRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.IsInMaintenanceModeRequest}
    */
   public  static final class IsInMaintenanceModeRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsInMaintenanceModeRequest)
       IsInMaintenanceModeRequestOrBuilder {
     // Use IsInMaintenanceModeRequest.newBuilder() to construct.
-    private IsInMaintenanceModeRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsInMaintenanceModeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsInMaintenanceModeRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsInMaintenanceModeRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -27984,22 +27984,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -28016,7 +28016,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -28060,61 +28060,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -28132,7 +28132,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -28140,15 +28140,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsInMaintenanceModeRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsInMaintenanceModeRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -28161,12 +28161,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -28175,7 +28175,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeRequest_descriptor;
       }
@@ -28202,29 +28202,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest)other);
         } else {
@@ -28245,13 +28245,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -28262,12 +28262,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -28285,22 +28285,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsInMaintenanceModeRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsInMaintenanceModeRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -28312,7 +28312,7 @@ public final class MasterProtos {
 
   public interface IsInMaintenanceModeResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsInMaintenanceModeResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool inMaintenanceMode = 1;
@@ -28327,11 +28327,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsInMaintenanceModeResponse}
    */
   public  static final class IsInMaintenanceModeResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsInMaintenanceModeResponse)
       IsInMaintenanceModeResponseOrBuilder {
     // Use IsInMaintenanceModeResponse.newBuilder() to construct.
-    private IsInMaintenanceModeResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsInMaintenanceModeResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsInMaintenanceModeResponse() {
@@ -28339,18 +28339,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsInMaintenanceModeResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -28373,22 +28373,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -28425,7 +28425,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, inMaintenanceMode_);
@@ -28439,7 +28439,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, inMaintenanceMode_);
       }
       size += unknownFields.getSerializedSize();
@@ -28477,7 +28477,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasInMaintenanceMode()) {
         hash = (37 * hash) + INMAINTENANCEMODE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getInMaintenanceMode());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -28486,61 +28486,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -28558,7 +28558,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -28566,15 +28566,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsInMaintenanceModeResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsInMaintenanceModeResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -28587,12 +28587,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -28603,7 +28603,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsInMaintenanceModeResponse_descriptor;
       }
@@ -28637,29 +28637,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse)other);
         } else {
@@ -28686,13 +28686,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -28736,12 +28736,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -28759,22 +28759,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsInMaintenanceModeResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsInMaintenanceModeResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -28786,7 +28786,7 @@ public final class MasterProtos {
 
   public interface BalanceRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BalanceRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool force = 1;
@@ -28801,11 +28801,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.BalanceRequest}
    */
   public  static final class BalanceRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BalanceRequest)
       BalanceRequestOrBuilder {
     // Use BalanceRequest.newBuilder() to construct.
-    private BalanceRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BalanceRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BalanceRequest() {
@@ -28813,18 +28813,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BalanceRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -28847,22 +28847,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -28895,7 +28895,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, force_);
@@ -28909,7 +28909,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, force_);
       }
       size += unknownFields.getSerializedSize();
@@ -28947,7 +28947,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasForce()) {
         hash = (37 * hash) + FORCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getForce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -28956,61 +28956,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -29028,7 +29028,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -29036,15 +29036,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.BalanceRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BalanceRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -29057,12 +29057,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -29073,7 +29073,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceRequest_descriptor;
       }
@@ -29107,29 +29107,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest)other);
         } else {
@@ -29153,13 +29153,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -29203,12 +29203,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -29226,22 +29226,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BalanceRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BalanceRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -29253,7 +29253,7 @@ public final class MasterProtos {
 
   public interface BalanceResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BalanceResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool balancer_ran = 1;
@@ -29268,11 +29268,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.BalanceResponse}
    */
   public  static final class BalanceResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BalanceResponse)
       BalanceResponseOrBuilder {
     // Use BalanceResponse.newBuilder() to construct.
-    private BalanceResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BalanceResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BalanceResponse() {
@@ -29280,18 +29280,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BalanceResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -29314,22 +29314,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -29366,7 +29366,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, balancerRan_);
@@ -29380,7 +29380,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, balancerRan_);
       }
       size += unknownFields.getSerializedSize();
@@ -29418,7 +29418,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasBalancerRan()) {
         hash = (37 * hash) + BALANCER_RAN_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getBalancerRan());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -29427,61 +29427,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -29499,7 +29499,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -29507,15 +29507,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.BalanceResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BalanceResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -29528,12 +29528,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -29544,7 +29544,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_BalanceResponse_descriptor;
       }
@@ -29578,29 +29578,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse)other);
         } else {
@@ -29627,13 +29627,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -29677,12 +29677,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -29700,22 +29700,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BalanceResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BalanceResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -29727,7 +29727,7 @@ public final class MasterProtos {
 
   public interface SetBalancerRunningRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetBalancerRunningRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool on = 1;
@@ -29751,11 +29751,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetBalancerRunningRequest}
    */
   public  static final class SetBalancerRunningRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetBalancerRunningRequest)
       SetBalancerRunningRequestOrBuilder {
     // Use SetBalancerRunningRequest.newBuilder() to construct.
-    private SetBalancerRunningRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetBalancerRunningRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetBalancerRunningRequest() {
@@ -29764,18 +29764,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetBalancerRunningRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -29803,22 +29803,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -29870,7 +29870,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, on_);
@@ -29887,11 +29887,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, on_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, synchronous_);
       }
       size += unknownFields.getSerializedSize();
@@ -29934,12 +29934,12 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasOn()) {
         hash = (37 * hash) + ON_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getOn());
       }
       if (hasSynchronous()) {
         hash = (37 * hash) + SYNCHRONOUS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getSynchronous());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -29948,61 +29948,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -30020,7 +30020,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -30028,15 +30028,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetBalancerRunningRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetBalancerRunningRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -30049,12 +30049,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -30067,7 +30067,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningRequest_descriptor;
       }
@@ -30105,29 +30105,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest)other);
         } else {
@@ -30157,13 +30157,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -30239,12 +30239,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -30262,22 +30262,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetBalancerRunningRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetBalancerRunningRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -30289,7 +30289,7 @@ public final class MasterProtos {
 
   public interface SetBalancerRunningResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetBalancerRunningResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool prev_balance_value = 1;
@@ -30304,11 +30304,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetBalancerRunningResponse}
    */
   public  static final class SetBalancerRunningResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetBalancerRunningResponse)
       SetBalancerRunningResponseOrBuilder {
     // Use SetBalancerRunningResponse.newBuilder() to construct.
-    private SetBalancerRunningResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetBalancerRunningResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetBalancerRunningResponse() {
@@ -30316,18 +30316,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetBalancerRunningResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -30350,22 +30350,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -30398,7 +30398,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, prevBalanceValue_);
@@ -30412,7 +30412,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, prevBalanceValue_);
       }
       size += unknownFields.getSerializedSize();
@@ -30450,7 +30450,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasPrevBalanceValue()) {
         hash = (37 * hash) + PREV_BALANCE_VALUE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getPrevBalanceValue());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -30459,61 +30459,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -30531,7 +30531,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -30539,15 +30539,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetBalancerRunningResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetBalancerRunningResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -30560,12 +30560,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -30576,7 +30576,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetBalancerRunningResponse_descriptor;
       }
@@ -30610,29 +30610,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse)other);
         } else {
@@ -30656,13 +30656,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -30706,12 +30706,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -30729,22 +30729,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetBalancerRunningResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetBalancerRunningResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -30756,34 +30756,34 @@ public final class MasterProtos {
 
   public interface IsBalancerEnabledRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsBalancerEnabledRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.IsBalancerEnabledRequest}
    */
   public  static final class IsBalancerEnabledRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsBalancerEnabledRequest)
       IsBalancerEnabledRequestOrBuilder {
     // Use IsBalancerEnabledRequest.newBuilder() to construct.
-    private IsBalancerEnabledRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsBalancerEnabledRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsBalancerEnabledRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsBalancerEnabledRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -30801,22 +30801,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -30833,7 +30833,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -30877,61 +30877,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -30949,7 +30949,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -30957,15 +30957,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsBalancerEnabledRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsBalancerEnabledRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -30978,12 +30978,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -30992,7 +30992,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledRequest_descriptor;
       }
@@ -31019,29 +31019,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest)other);
         } else {
@@ -31062,13 +31062,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -31079,12 +31079,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -31102,22 +31102,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsBalancerEnabledRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsBalancerEnabledRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -31129,7 +31129,7 @@ public final class MasterProtos {
 
   public interface IsBalancerEnabledResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsBalancerEnabledResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool enabled = 1;
@@ -31144,11 +31144,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsBalancerEnabledResponse}
    */
   public  static final class IsBalancerEnabledResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsBalancerEnabledResponse)
       IsBalancerEnabledResponseOrBuilder {
     // Use IsBalancerEnabledResponse.newBuilder() to construct.
-    private IsBalancerEnabledResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsBalancerEnabledResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsBalancerEnabledResponse() {
@@ -31156,18 +31156,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsBalancerEnabledResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -31190,22 +31190,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -31242,7 +31242,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, enabled_);
@@ -31256,7 +31256,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, enabled_);
       }
       size += unknownFields.getSerializedSize();
@@ -31294,7 +31294,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getEnabled());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -31303,61 +31303,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -31375,7 +31375,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -31383,15 +31383,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsBalancerEnabledResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsBalancerEnabledResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -31404,12 +31404,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -31420,7 +31420,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsBalancerEnabledResponse_descriptor;
       }
@@ -31454,29 +31454,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse)other);
         } else {
@@ -31503,13 +31503,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -31553,12 +31553,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -31576,22 +31576,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsBalancerEnabledResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsBalancerEnabledResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -31603,7 +31603,7 @@ public final class MasterProtos {
 
   public interface SetSplitOrMergeEnabledRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetSplitOrMergeEnabledRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool enabled = 1;
@@ -31640,11 +31640,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledRequest}
    */
   public  static final class SetSplitOrMergeEnabledRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetSplitOrMergeEnabledRequest)
       SetSplitOrMergeEnabledRequestOrBuilder {
     // Use SetSplitOrMergeEnabledRequest.newBuilder() to construct.
-    private SetSplitOrMergeEnabledRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetSplitOrMergeEnabledRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetSplitOrMergeEnabledRequest() {
@@ -31654,18 +31654,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetSplitOrMergeEnabledRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -31726,10 +31726,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -31739,12 +31739,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -31784,9 +31784,9 @@ public final class MasterProtos {
 
     public static final int SWITCH_TYPES_FIELD_NUMBER = 3;
     private java.util.List switchTypes_;
-    private static final com.google.protobuf.Internal.ListAdapter.Converter<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter.Converter<
         java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType> switchTypes_converter_ =
-            new com.google.protobuf.Internal.ListAdapter.Converter<
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter.Converter<
                 java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType>() {
               public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType convert(java.lang.Integer from) {
                 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType result = org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(from);
@@ -31797,7 +31797,7 @@ public final class MasterProtos {
      * repeated .hbase.pb.MasterSwitchType switch_types = 3;
      */
     public java.util.List getSwitchTypesList() {
-      return new com.google.protobuf.Internal.ListAdapter<
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter<
           java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType>(switchTypes_, switchTypes_converter_);
     }
     /**
@@ -31827,7 +31827,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, enabled_);
@@ -31847,17 +31847,17 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, enabled_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, synchronous_);
       }
       {
         int dataSize = 0;
         for (int i = 0; i < switchTypes_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeEnumSizeNoTag(switchTypes_.get(i));
         }
         size += dataSize;
@@ -31904,12 +31904,12 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getEnabled());
       }
       if (hasSynchronous()) {
         hash = (37 * hash) + SYNCHRONOUS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getSynchronous());
       }
       if (getSwitchTypesCount() > 0) {
@@ -31922,61 +31922,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -31994,7 +31994,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -32002,15 +32002,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetSplitOrMergeEnabledRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -32023,12 +32023,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -32043,7 +32043,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
       }
@@ -32086,29 +32086,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)other);
         } else {
@@ -32148,13 +32148,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -32242,7 +32242,7 @@ public final class MasterProtos {
        * repeated .hbase.pb.MasterSwitchType switch_types = 3;
        */
       public java.util.List getSwitchTypesList() {
-        return new com.google.protobuf.Internal.ListAdapter<
+        return new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter<
             java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterSwitchType>(switchTypes_, switchTypes_converter_);
       }
       /**
@@ -32304,12 +32304,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -32327,22 +32327,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetSplitOrMergeEnabledRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetSplitOrMergeEnabledRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -32354,7 +32354,7 @@ public final class MasterProtos {
 
   public interface SetSplitOrMergeEnabledResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetSplitOrMergeEnabledResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated bool prev_value = 1;
@@ -32373,11 +32373,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledResponse}
    */
   public  static final class SetSplitOrMergeEnabledResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetSplitOrMergeEnabledResponse)
       SetSplitOrMergeEnabledResponseOrBuilder {
     // Use SetSplitOrMergeEnabledResponse.newBuilder() to construct.
-    private SetSplitOrMergeEnabledResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetSplitOrMergeEnabledResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetSplitOrMergeEnabledResponse() {
@@ -32385,18 +32385,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetSplitOrMergeEnabledResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -32435,10 +32435,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -32448,12 +32448,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -32492,7 +32492,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < prevValue_.size(); i++) {
         output.writeBool(1, prevValue_.get(i));
@@ -32551,61 +32551,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -32623,7 +32623,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -32631,15 +32631,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetSplitOrMergeEnabledResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -32652,12 +32652,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -32668,7 +32668,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
       }
@@ -32701,29 +32701,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse)other);
         } else {
@@ -32754,13 +32754,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -32823,7 +32823,7 @@ public final class MasterProtos {
       public Builder addAllPrevValue(
           java.lang.Iterable values) {
         ensurePrevValueIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, prevValue_);
         onChanged();
         return this;
@@ -32838,12 +32838,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -32861,22 +32861,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetSplitOrMergeEnabledResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetSplitOrMergeEnabledResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -32888,7 +32888,7 @@ public final class MasterProtos {
 
   public interface IsSplitOrMergeEnabledRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsSplitOrMergeEnabledRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.MasterSwitchType switch_type = 1;
@@ -32903,11 +32903,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledRequest}
    */
   public  static final class IsSplitOrMergeEnabledRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsSplitOrMergeEnabledRequest)
       IsSplitOrMergeEnabledRequestOrBuilder {
     // Use IsSplitOrMergeEnabledRequest.newBuilder() to construct.
-    private IsSplitOrMergeEnabledRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsSplitOrMergeEnabledRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsSplitOrMergeEnabledRequest() {
@@ -32915,18 +32915,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsSplitOrMergeEnabledRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -32955,22 +32955,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -33008,7 +33008,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, switchType_);
@@ -33022,7 +33022,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, switchType_);
       }
       size += unknownFields.getSerializedSize();
@@ -33067,61 +33067,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -33139,7 +33139,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -33147,15 +33147,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsSplitOrMergeEnabledRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -33168,12 +33168,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -33184,7 +33184,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
       }
@@ -33218,29 +33218,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)other);
         } else {
@@ -33267,13 +33267,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -33321,12 +33321,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -33344,22 +33344,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsSplitOrMergeEnabledRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsSplitOrMergeEnabledRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -33371,7 +33371,7 @@ public final class MasterProtos {
 
   public interface IsSplitOrMergeEnabledResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsSplitOrMergeEnabledResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool enabled = 1;
@@ -33386,11 +33386,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledResponse}
    */
   public  static final class IsSplitOrMergeEnabledResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsSplitOrMergeEnabledResponse)
       IsSplitOrMergeEnabledResponseOrBuilder {
     // Use IsSplitOrMergeEnabledResponse.newBuilder() to construct.
-    private IsSplitOrMergeEnabledResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsSplitOrMergeEnabledResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsSplitOrMergeEnabledResponse() {
@@ -33398,18 +33398,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsSplitOrMergeEnabledResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -33432,22 +33432,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -33484,7 +33484,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, enabled_);
@@ -33498,7 +33498,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, enabled_);
       }
       size += unknownFields.getSerializedSize();
@@ -33536,7 +33536,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getEnabled());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -33545,61 +33545,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -33617,7 +33617,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -33625,15 +33625,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsSplitOrMergeEnabledResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -33646,12 +33646,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -33662,7 +33662,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_descriptor;
       }
@@ -33696,29 +33696,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse)other);
         } else {
@@ -33745,13 +33745,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -33795,12 +33795,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -33818,22 +33818,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsSplitOrMergeEnabledResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsSplitOrMergeEnabledResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -33845,34 +33845,34 @@ public final class MasterProtos {
 
   public interface NormalizeRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NormalizeRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.NormalizeRequest}
    */
   public  static final class NormalizeRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NormalizeRequest)
       NormalizeRequestOrBuilder {
     // Use NormalizeRequest.newBuilder() to construct.
-    private NormalizeRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NormalizeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NormalizeRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NormalizeRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -33890,22 +33890,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -33922,7 +33922,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -33966,61 +33966,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -34038,7 +34038,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -34046,15 +34046,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.NormalizeRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NormalizeRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -34067,12 +34067,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -34081,7 +34081,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
       }
@@ -34108,29 +34108,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest)other);
         } else {
@@ -34151,13 +34151,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -34168,12 +34168,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -34191,22 +34191,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NormalizeRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NormalizeRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -34218,7 +34218,7 @@ public final class MasterProtos {
 
   public interface NormalizeResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.NormalizeResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool normalizer_ran = 1;
@@ -34233,11 +34233,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.NormalizeResponse}
    */
   public  static final class NormalizeResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.NormalizeResponse)
       NormalizeResponseOrBuilder {
     // Use NormalizeResponse.newBuilder() to construct.
-    private NormalizeResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private NormalizeResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private NormalizeResponse() {
@@ -34245,18 +34245,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private NormalizeResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -34279,22 +34279,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -34331,7 +34331,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, normalizerRan_);
@@ -34345,7 +34345,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, normalizerRan_);
       }
       size += unknownFields.getSerializedSize();
@@ -34383,7 +34383,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasNormalizerRan()) {
         hash = (37 * hash) + NORMALIZER_RAN_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getNormalizerRan());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -34392,61 +34392,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -34464,7 +34464,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -34472,15 +34472,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.NormalizeResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.NormalizeResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -34493,12 +34493,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -34509,7 +34509,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
       }
@@ -34543,29 +34543,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse)other);
         } else {
@@ -34592,13 +34592,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -34642,12 +34642,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -34665,22 +34665,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public NormalizeResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new NormalizeResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -34692,7 +34692,7 @@ public final class MasterProtos {
 
   public interface SetNormalizerRunningRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetNormalizerRunningRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool on = 1;
@@ -34707,11 +34707,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetNormalizerRunningRequest}
    */
   public  static final class SetNormalizerRunningRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetNormalizerRunningRequest)
       SetNormalizerRunningRequestOrBuilder {
     // Use SetNormalizerRunningRequest.newBuilder() to construct.
-    private SetNormalizerRunningRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetNormalizerRunningRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetNormalizerRunningRequest() {
@@ -34719,18 +34719,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetNormalizerRunningRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -34753,22 +34753,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -34805,7 +34805,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, on_);
@@ -34819,7 +34819,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, on_);
       }
       size += unknownFields.getSerializedSize();
@@ -34857,7 +34857,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasOn()) {
         hash = (37 * hash) + ON_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getOn());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -34866,61 +34866,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -34938,7 +34938,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -34946,15 +34946,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetNormalizerRunningRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetNormalizerRunningRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -34967,12 +34967,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -34983,7 +34983,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
       }
@@ -35017,29 +35017,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)other);
         } else {
@@ -35066,13 +35066,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -35116,12 +35116,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -35139,22 +35139,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetNormalizerRunningRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetNormalizerRunningRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -35166,7 +35166,7 @@ public final class MasterProtos {
 
   public interface SetNormalizerRunningResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetNormalizerRunningResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool prev_normalizer_value = 1;
@@ -35181,11 +35181,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetNormalizerRunningResponse}
    */
   public  static final class SetNormalizerRunningResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetNormalizerRunningResponse)
       SetNormalizerRunningResponseOrBuilder {
     // Use SetNormalizerRunningResponse.newBuilder() to construct.
-    private SetNormalizerRunningResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetNormalizerRunningResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetNormalizerRunningResponse() {
@@ -35193,18 +35193,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetNormalizerRunningResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -35227,22 +35227,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -35275,7 +35275,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, prevNormalizerValue_);
@@ -35289,7 +35289,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, prevNormalizerValue_);
       }
       size += unknownFields.getSerializedSize();
@@ -35327,7 +35327,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasPrevNormalizerValue()) {
         hash = (37 * hash) + PREV_NORMALIZER_VALUE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getPrevNormalizerValue());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -35336,61 +35336,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -35408,7 +35408,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -35416,15 +35416,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetNormalizerRunningResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetNormalizerRunningResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -35437,12 +35437,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -35453,7 +35453,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningResponse_descriptor;
       }
@@ -35487,29 +35487,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse)other);
         } else {
@@ -35533,13 +35533,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -35583,12 +35583,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -35606,22 +35606,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetNormalizerRunningResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetNormalizerRunningResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -35633,34 +35633,34 @@ public final class MasterProtos {
 
   public interface IsNormalizerEnabledRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsNormalizerEnabledRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.IsNormalizerEnabledRequest}
    */
   public  static final class IsNormalizerEnabledRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsNormalizerEnabledRequest)
       IsNormalizerEnabledRequestOrBuilder {
     // Use IsNormalizerEnabledRequest.newBuilder() to construct.
-    private IsNormalizerEnabledRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsNormalizerEnabledRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsNormalizerEnabledRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsNormalizerEnabledRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -35678,22 +35678,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -35710,7 +35710,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -35754,61 +35754,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -35826,7 +35826,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -35834,15 +35834,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsNormalizerEnabledRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsNormalizerEnabledRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -35855,12 +35855,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -35869,7 +35869,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledRequest_descriptor;
       }
@@ -35896,29 +35896,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest)other);
         } else {
@@ -35939,13 +35939,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -35956,12 +35956,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -35979,22 +35979,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsNormalizerEnabledRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsNormalizerEnabledRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -36006,7 +36006,7 @@ public final class MasterProtos {
 
   public interface IsNormalizerEnabledResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsNormalizerEnabledResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool enabled = 1;
@@ -36021,11 +36021,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsNormalizerEnabledResponse}
    */
   public  static final class IsNormalizerEnabledResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsNormalizerEnabledResponse)
       IsNormalizerEnabledResponseOrBuilder {
     // Use IsNormalizerEnabledResponse.newBuilder() to construct.
-    private IsNormalizerEnabledResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsNormalizerEnabledResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsNormalizerEnabledResponse() {
@@ -36033,18 +36033,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsNormalizerEnabledResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -36067,22 +36067,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -36119,7 +36119,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, enabled_);
@@ -36133,7 +36133,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, enabled_);
       }
       size += unknownFields.getSerializedSize();
@@ -36171,7 +36171,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getEnabled());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -36180,61 +36180,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -36252,7 +36252,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -36260,15 +36260,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsNormalizerEnabledResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsNormalizerEnabledResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -36281,12 +36281,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -36297,7 +36297,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsNormalizerEnabledResponse_descriptor;
       }
@@ -36331,29 +36331,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse)other);
         } else {
@@ -36380,13 +36380,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -36430,12 +36430,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -36453,22 +36453,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsNormalizerEnabledResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsNormalizerEnabledResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -36480,34 +36480,34 @@ public final class MasterProtos {
 
   public interface RunCatalogScanRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RunCatalogScanRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.RunCatalogScanRequest}
    */
   public  static final class RunCatalogScanRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RunCatalogScanRequest)
       RunCatalogScanRequestOrBuilder {
     // Use RunCatalogScanRequest.newBuilder() to construct.
-    private RunCatalogScanRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RunCatalogScanRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RunCatalogScanRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RunCatalogScanRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -36525,22 +36525,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -36557,7 +36557,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -36601,61 +36601,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -36673,7 +36673,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -36681,15 +36681,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.RunCatalogScanRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RunCatalogScanRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -36702,12 +36702,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -36716,7 +36716,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanRequest_descriptor;
       }
@@ -36743,29 +36743,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest)other);
         } else {
@@ -36786,13 +36786,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -36803,12 +36803,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -36826,22 +36826,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RunCatalogScanRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RunCatalogScanRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -36853,7 +36853,7 @@ public final class MasterProtos {
 
   public interface RunCatalogScanResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RunCatalogScanResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional int32 scan_result = 1;
@@ -36868,11 +36868,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.RunCatalogScanResponse}
    */
   public  static final class RunCatalogScanResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RunCatalogScanResponse)
       RunCatalogScanResponseOrBuilder {
     // Use RunCatalogScanResponse.newBuilder() to construct.
-    private RunCatalogScanResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RunCatalogScanResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RunCatalogScanResponse() {
@@ -36880,18 +36880,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RunCatalogScanResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -36914,22 +36914,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -36962,7 +36962,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(1, scanResult_);
@@ -36976,7 +36976,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(1, scanResult_);
       }
       size += unknownFields.getSerializedSize();
@@ -37022,61 +37022,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -37094,7 +37094,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -37102,15 +37102,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.RunCatalogScanResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RunCatalogScanResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -37123,12 +37123,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -37139,7 +37139,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RunCatalogScanResponse_descriptor;
       }
@@ -37173,29 +37173,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse)other);
         } else {
@@ -37219,13 +37219,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -37269,12 +37269,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -37292,22 +37292,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RunCatalogScanResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RunCatalogScanResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -37319,7 +37319,7 @@ public final class MasterProtos {
 
   public interface EnableCatalogJanitorRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.EnableCatalogJanitorRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool enable = 1;
@@ -37334,11 +37334,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.EnableCatalogJanitorRequest}
    */
   public  static final class EnableCatalogJanitorRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.EnableCatalogJanitorRequest)
       EnableCatalogJanitorRequestOrBuilder {
     // Use EnableCatalogJanitorRequest.newBuilder() to construct.
-    private EnableCatalogJanitorRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private EnableCatalogJanitorRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private EnableCatalogJanitorRequest() {
@@ -37346,18 +37346,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private EnableCatalogJanitorRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -37380,22 +37380,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -37432,7 +37432,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, enable_);
@@ -37446,7 +37446,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, enable_);
       }
       size += unknownFields.getSerializedSize();
@@ -37484,7 +37484,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasEnable()) {
         hash = (37 * hash) + ENABLE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getEnable());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -37493,61 +37493,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -37565,7 +37565,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -37573,15 +37573,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.EnableCatalogJanitorRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.EnableCatalogJanitorRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -37594,12 +37594,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -37610,7 +37610,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorRequest_descriptor;
       }
@@ -37644,29 +37644,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest)other);
         } else {
@@ -37693,13 +37693,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -37743,12 +37743,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -37766,22 +37766,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public EnableCatalogJanitorRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new EnableCatalogJanitorRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -37793,7 +37793,7 @@ public final class MasterProtos {
 
   public interface EnableCatalogJanitorResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.EnableCatalogJanitorResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool prev_value = 1;
@@ -37808,11 +37808,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.EnableCatalogJanitorResponse}
    */
   public  static final class EnableCatalogJanitorResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.EnableCatalogJanitorResponse)
       EnableCatalogJanitorResponseOrBuilder {
     // Use EnableCatalogJanitorResponse.newBuilder() to construct.
-    private EnableCatalogJanitorResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private EnableCatalogJanitorResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private EnableCatalogJanitorResponse() {
@@ -37820,18 +37820,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private EnableCatalogJanitorResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -37854,22 +37854,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -37902,7 +37902,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, prevValue_);
@@ -37916,7 +37916,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, prevValue_);
       }
       size += unknownFields.getSerializedSize();
@@ -37954,7 +37954,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasPrevValue()) {
         hash = (37 * hash) + PREV_VALUE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getPrevValue());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -37963,61 +37963,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -38035,7 +38035,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -38043,15 +38043,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.EnableCatalogJanitorResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.EnableCatalogJanitorResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -38064,12 +38064,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -38080,7 +38080,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_EnableCatalogJanitorResponse_descriptor;
       }
@@ -38114,29 +38114,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse)other);
         } else {
@@ -38160,13 +38160,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -38210,12 +38210,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -38233,22 +38233,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public EnableCatalogJanitorResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new EnableCatalogJanitorResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -38260,34 +38260,34 @@ public final class MasterProtos {
 
   public interface IsCatalogJanitorEnabledRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsCatalogJanitorEnabledRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.IsCatalogJanitorEnabledRequest}
    */
   public  static final class IsCatalogJanitorEnabledRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsCatalogJanitorEnabledRequest)
       IsCatalogJanitorEnabledRequestOrBuilder {
     // Use IsCatalogJanitorEnabledRequest.newBuilder() to construct.
-    private IsCatalogJanitorEnabledRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsCatalogJanitorEnabledRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsCatalogJanitorEnabledRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsCatalogJanitorEnabledRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -38305,22 +38305,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -38337,7 +38337,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -38381,61 +38381,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -38453,7 +38453,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -38461,15 +38461,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsCatalogJanitorEnabledRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsCatalogJanitorEnabledRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -38482,12 +38482,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -38496,7 +38496,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_descriptor;
       }
@@ -38523,29 +38523,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest)other);
         } else {
@@ -38566,13 +38566,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -38583,12 +38583,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -38606,22 +38606,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsCatalogJanitorEnabledRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsCatalogJanitorEnabledRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -38633,7 +38633,7 @@ public final class MasterProtos {
 
   public interface IsCatalogJanitorEnabledResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsCatalogJanitorEnabledResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool value = 1;
@@ -38648,11 +38648,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsCatalogJanitorEnabledResponse}
    */
   public  static final class IsCatalogJanitorEnabledResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsCatalogJanitorEnabledResponse)
       IsCatalogJanitorEnabledResponseOrBuilder {
     // Use IsCatalogJanitorEnabledResponse.newBuilder() to construct.
-    private IsCatalogJanitorEnabledResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsCatalogJanitorEnabledResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsCatalogJanitorEnabledResponse() {
@@ -38660,18 +38660,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsCatalogJanitorEnabledResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -38694,22 +38694,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -38746,7 +38746,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, value_);
@@ -38760,7 +38760,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, value_);
       }
       size += unknownFields.getSerializedSize();
@@ -38798,7 +38798,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasValue()) {
         hash = (37 * hash) + VALUE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getValue());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -38807,61 +38807,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -38879,7 +38879,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -38887,15 +38887,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsCatalogJanitorEnabledResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsCatalogJanitorEnabledResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -38908,12 +38908,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -38924,7 +38924,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_descriptor;
       }
@@ -38958,29 +38958,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse)other);
         } else {
@@ -39007,13 +39007,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -39057,12 +39057,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -39080,22 +39080,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsCatalogJanitorEnabledResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsCatalogJanitorEnabledResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -39107,7 +39107,7 @@ public final class MasterProtos {
 
   public interface SnapshotRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.SnapshotDescription snapshot = 1;
@@ -39126,29 +39126,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SnapshotRequest}
    */
   public  static final class SnapshotRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotRequest)
       SnapshotRequestOrBuilder {
     // Use SnapshotRequest.newBuilder() to construct.
-    private SnapshotRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SnapshotRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SnapshotRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SnapshotRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -39179,22 +39179,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -39241,7 +39241,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSnapshot());
@@ -39255,7 +39255,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSnapshot());
       }
       size += unknownFields.getSerializedSize();
@@ -39301,61 +39301,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -39373,7 +39373,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -39381,15 +39381,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SnapshotRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -39402,12 +39402,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -39423,7 +39423,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotRequest_descriptor;
       }
@@ -39461,29 +39461,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest)other);
         } else {
@@ -39513,13 +39513,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -39532,7 +39532,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 1;
@@ -39635,11 +39635,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -39649,12 +39649,12 @@ public final class MasterProtos {
         return snapshotBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -39672,22 +39672,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SnapshotRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SnapshotRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -39699,7 +39699,7 @@ public final class MasterProtos {
 
   public interface SnapshotResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int64 expected_timeout = 1;
@@ -39714,11 +39714,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SnapshotResponse}
    */
   public  static final class SnapshotResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotResponse)
       SnapshotResponseOrBuilder {
     // Use SnapshotResponse.newBuilder() to construct.
-    private SnapshotResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SnapshotResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SnapshotResponse() {
@@ -39726,18 +39726,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SnapshotResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -39760,22 +39760,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -39812,7 +39812,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, expectedTimeout_);
@@ -39826,7 +39826,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, expectedTimeout_);
       }
       size += unknownFields.getSerializedSize();
@@ -39864,7 +39864,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasExpectedTimeout()) {
         hash = (37 * hash) + EXPECTED_TIMEOUT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getExpectedTimeout());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -39873,61 +39873,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -39945,7 +39945,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -39953,15 +39953,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SnapshotResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -39974,12 +39974,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -39990,7 +39990,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SnapshotResponse_descriptor;
       }
@@ -40024,29 +40024,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse)other);
         } else {
@@ -40073,13 +40073,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -40123,12 +40123,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -40146,22 +40146,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SnapshotResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SnapshotResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -40173,34 +40173,34 @@ public final class MasterProtos {
 
   public interface GetCompletedSnapshotsRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetCompletedSnapshotsRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.GetCompletedSnapshotsRequest}
    */
   public  static final class GetCompletedSnapshotsRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetCompletedSnapshotsRequest)
       GetCompletedSnapshotsRequestOrBuilder {
     // Use GetCompletedSnapshotsRequest.newBuilder() to construct.
-    private GetCompletedSnapshotsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetCompletedSnapshotsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetCompletedSnapshotsRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetCompletedSnapshotsRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -40218,22 +40218,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -40250,7 +40250,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -40294,61 +40294,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -40366,7 +40366,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -40374,15 +40374,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetCompletedSnapshotsRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetCompletedSnapshotsRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -40395,12 +40395,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -40409,7 +40409,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsRequest_descriptor;
       }
@@ -40436,29 +40436,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest)other);
         } else {
@@ -40479,13 +40479,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -40496,12 +40496,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -40519,22 +40519,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetCompletedSnapshotsRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetCompletedSnapshotsRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -40546,7 +40546,7 @@ public final class MasterProtos {
 
   public interface GetCompletedSnapshotsResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetCompletedSnapshotsResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.SnapshotDescription snapshots = 1;
@@ -40576,11 +40576,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetCompletedSnapshotsResponse}
    */
   public  static final class GetCompletedSnapshotsResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetCompletedSnapshotsResponse)
       GetCompletedSnapshotsResponseOrBuilder {
     // Use GetCompletedSnapshotsResponse.newBuilder() to construct.
-    private GetCompletedSnapshotsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetCompletedSnapshotsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetCompletedSnapshotsResponse() {
@@ -40588,18 +40588,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetCompletedSnapshotsResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -40626,10 +40626,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -40639,12 +40639,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -40702,7 +40702,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < snapshots_.size(); i++) {
         output.writeMessage(1, snapshots_.get(i));
@@ -40716,7 +40716,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < snapshots_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, snapshots_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -40759,61 +40759,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -40831,7 +40831,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -40839,15 +40839,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetCompletedSnapshotsResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetCompletedSnapshotsResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -40860,12 +40860,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotsFieldBuilder();
         }
@@ -40881,7 +40881,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetCompletedSnapshotsResponse_descriptor;
       }
@@ -40918,29 +40918,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse)other);
         } else {
@@ -40970,7 +40970,7 @@ public final class MasterProtos {
               snapshots_ = other.snapshots_;
               bitField0_ = (bitField0_ & ~0x00000001);
               snapshotsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSnapshotsFieldBuilder() : null;
             } else {
               snapshotsBuilder_.addAllMessages(other.snapshots_);
@@ -40992,13 +40992,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -41019,7 +41019,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotsBuilder_;
 
       /**
@@ -41151,7 +41151,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (snapshotsBuilder_ == null) {
           ensureSnapshotsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, snapshots_);
           onChanged();
         } else {
@@ -41235,11 +41235,11 @@ public final class MasterProtos {
            getSnapshotsBuilderList() {
         return getSnapshotsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotsFieldBuilder() {
         if (snapshotsBuilder_ == null) {
-          snapshotsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          snapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   snapshots_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -41250,12 +41250,12 @@ public final class MasterProtos {
         return snapshotsBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -41273,22 +41273,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetCompletedSnapshotsResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetCompletedSnapshotsResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -41300,7 +41300,7 @@ public final class MasterProtos {
 
   public interface DeleteSnapshotRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteSnapshotRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.SnapshotDescription snapshot = 1;
@@ -41319,29 +41319,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.DeleteSnapshotRequest}
    */
   public  static final class DeleteSnapshotRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteSnapshotRequest)
       DeleteSnapshotRequestOrBuilder {
     // Use DeleteSnapshotRequest.newBuilder() to construct.
-    private DeleteSnapshotRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteSnapshotRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteSnapshotRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteSnapshotRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -41372,22 +41372,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -41434,7 +41434,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSnapshot());
@@ -41448,7 +41448,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSnapshot());
       }
       size += unknownFields.getSerializedSize();
@@ -41494,61 +41494,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -41566,7 +41566,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -41574,15 +41574,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteSnapshotRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteSnapshotRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -41595,12 +41595,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -41616,7 +41616,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotRequest_descriptor;
       }
@@ -41654,29 +41654,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest)other);
         } else {
@@ -41706,13 +41706,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -41725,7 +41725,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 1;
@@ -41828,11 +41828,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -41842,12 +41842,12 @@ public final class MasterProtos {
         return snapshotBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -41865,22 +41865,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteSnapshotRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteSnapshotRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -41892,34 +41892,34 @@ public final class MasterProtos {
 
   public interface DeleteSnapshotResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeleteSnapshotResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.DeleteSnapshotResponse}
    */
   public  static final class DeleteSnapshotResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeleteSnapshotResponse)
       DeleteSnapshotResponseOrBuilder {
     // Use DeleteSnapshotResponse.newBuilder() to construct.
-    private DeleteSnapshotResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeleteSnapshotResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeleteSnapshotResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeleteSnapshotResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -41937,22 +41937,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -41969,7 +41969,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -42013,61 +42013,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -42085,7 +42085,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -42093,15 +42093,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.DeleteSnapshotResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeleteSnapshotResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -42114,12 +42114,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -42128,7 +42128,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DeleteSnapshotResponse_descriptor;
       }
@@ -42155,29 +42155,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse)other);
         } else {
@@ -42198,13 +42198,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -42215,12 +42215,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -42238,22 +42238,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeleteSnapshotResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeleteSnapshotResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -42265,7 +42265,7 @@ public final class MasterProtos {
 
   public interface RestoreSnapshotRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RestoreSnapshotRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.SnapshotDescription snapshot = 1;
@@ -42302,11 +42302,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.RestoreSnapshotRequest}
    */
   public  static final class RestoreSnapshotRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RestoreSnapshotRequest)
       RestoreSnapshotRequestOrBuilder {
     // Use RestoreSnapshotRequest.newBuilder() to construct.
-    private RestoreSnapshotRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RestoreSnapshotRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RestoreSnapshotRequest() {
@@ -42315,18 +42315,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RestoreSnapshotRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -42367,22 +42367,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -42459,7 +42459,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSnapshot());
@@ -42479,15 +42479,15 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSnapshot());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, nonceGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -42539,12 +42539,12 @@ public final class MasterProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -42553,61 +42553,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -42625,7 +42625,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -42633,15 +42633,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.RestoreSnapshotRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RestoreSnapshotRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -42654,12 +42654,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -42679,7 +42679,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotRequest_descriptor;
       }
@@ -42725,29 +42725,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest)other);
         } else {
@@ -42783,13 +42783,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -42802,7 +42802,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 1;
@@ -42905,11 +42905,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.SnapshotDescription snapshot = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -42983,12 +42983,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -43006,22 +43006,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RestoreSnapshotRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RestoreSnapshotRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -43033,7 +43033,7 @@ public final class MasterProtos {
 
   public interface RestoreSnapshotResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RestoreSnapshotResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 proc_id = 1;
@@ -43048,11 +43048,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.RestoreSnapshotResponse}
    */
   public  static final class RestoreSnapshotResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RestoreSnapshotResponse)
       RestoreSnapshotResponseOrBuilder {
     // Use RestoreSnapshotResponse.newBuilder() to construct.
-    private RestoreSnapshotResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RestoreSnapshotResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RestoreSnapshotResponse() {
@@ -43060,18 +43060,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RestoreSnapshotResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -43094,22 +43094,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -43146,7 +43146,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -43160,7 +43160,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -43198,7 +43198,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -43207,61 +43207,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -43279,7 +43279,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -43287,15 +43287,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.RestoreSnapshotResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RestoreSnapshotResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -43308,12 +43308,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -43324,7 +43324,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RestoreSnapshotResponse_descriptor;
       }
@@ -43358,29 +43358,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse)other);
         } else {
@@ -43407,13 +43407,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -43457,12 +43457,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -43480,22 +43480,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RestoreSnapshotResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RestoreSnapshotResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -43507,7 +43507,7 @@ public final class MasterProtos {
 
   public interface IsSnapshotDoneRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsSnapshotDoneRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.SnapshotDescription snapshot = 1;
@@ -43531,29 +43531,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsSnapshotDoneRequest}
    */
   public  static final class IsSnapshotDoneRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsSnapshotDoneRequest)
       IsSnapshotDoneRequestOrBuilder {
     // Use IsSnapshotDoneRequest.newBuilder() to construct.
-    private IsSnapshotDoneRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsSnapshotDoneRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsSnapshotDoneRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsSnapshotDoneRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -43584,22 +43584,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -43644,7 +43644,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSnapshot());
@@ -43658,7 +43658,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSnapshot());
       }
       size += unknownFields.getSerializedSize();
@@ -43704,61 +43704,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -43776,7 +43776,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -43789,15 +43789,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsSnapshotDoneRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsSnapshotDoneRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -43810,12 +43810,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -43831,7 +43831,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneRequest_descriptor;
       }
@@ -43869,29 +43869,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest)other);
         } else {
@@ -43920,13 +43920,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -43939,7 +43939,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * optional .hbase.pb.SnapshotDescription snapshot = 1;
@@ -44042,11 +44042,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.SnapshotDescription snapshot = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -44056,12 +44056,12 @@ public final class MasterProtos {
         return snapshotBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -44079,22 +44079,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsSnapshotDoneRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsSnapshotDoneRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -44106,7 +44106,7 @@ public final class MasterProtos {
 
   public interface IsSnapshotDoneResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsSnapshotDoneResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool done = 1 [default = false];
@@ -44134,11 +44134,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsSnapshotDoneResponse}
    */
   public  static final class IsSnapshotDoneResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsSnapshotDoneResponse)
       IsSnapshotDoneResponseOrBuilder {
     // Use IsSnapshotDoneResponse.newBuilder() to construct.
-    private IsSnapshotDoneResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsSnapshotDoneResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsSnapshotDoneResponse() {
@@ -44146,18 +44146,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsSnapshotDoneResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -44193,22 +44193,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -44268,7 +44268,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, done_);
@@ -44285,11 +44285,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, done_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getSnapshot());
       }
       size += unknownFields.getSerializedSize();
@@ -44332,7 +44332,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasDone()) {
         hash = (37 * hash) + DONE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDone());
       }
       if (hasSnapshot()) {
@@ -44345,61 +44345,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -44417,7 +44417,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -44425,15 +44425,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsSnapshotDoneResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsSnapshotDoneResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -44446,12 +44446,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -44469,7 +44469,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSnapshotDoneResponse_descriptor;
       }
@@ -44511,29 +44511,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse)other);
         } else {
@@ -44565,13 +44565,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -44616,7 +44616,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * optional .hbase.pb.SnapshotDescription snapshot = 2;
@@ -44719,11 +44719,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.SnapshotDescription snapshot = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -44733,12 +44733,12 @@ public final class MasterProtos {
         return snapshotBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -44756,22 +44756,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsSnapshotDoneResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsSnapshotDoneResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -44783,7 +44783,7 @@ public final class MasterProtos {
 
   public interface IsRestoreSnapshotDoneRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsRestoreSnapshotDoneRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.SnapshotDescription snapshot = 1;
@@ -44802,29 +44802,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsRestoreSnapshotDoneRequest}
    */
   public  static final class IsRestoreSnapshotDoneRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsRestoreSnapshotDoneRequest)
       IsRestoreSnapshotDoneRequestOrBuilder {
     // Use IsRestoreSnapshotDoneRequest.newBuilder() to construct.
-    private IsRestoreSnapshotDoneRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsRestoreSnapshotDoneRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsRestoreSnapshotDoneRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsRestoreSnapshotDoneRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -44855,22 +44855,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -44915,7 +44915,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getSnapshot());
@@ -44929,7 +44929,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getSnapshot());
       }
       size += unknownFields.getSerializedSize();
@@ -44975,61 +44975,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -45047,7 +45047,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -45055,15 +45055,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsRestoreSnapshotDoneRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsRestoreSnapshotDoneRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -45076,12 +45076,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -45097,7 +45097,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_descriptor;
       }
@@ -45135,29 +45135,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest)other);
         } else {
@@ -45186,13 +45186,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -45205,7 +45205,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> snapshotBuilder_;
       /**
        * optional .hbase.pb.SnapshotDescription snapshot = 1;
@@ -45308,11 +45308,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.SnapshotDescription snapshot = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -45322,12 +45322,12 @@ public final class MasterProtos {
         return snapshotBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -45345,22 +45345,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsRestoreSnapshotDoneRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsRestoreSnapshotDoneRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -45372,7 +45372,7 @@ public final class MasterProtos {
 
   public interface IsRestoreSnapshotDoneResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsRestoreSnapshotDoneResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool done = 1 [default = false];
@@ -45387,11 +45387,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsRestoreSnapshotDoneResponse}
    */
   public  static final class IsRestoreSnapshotDoneResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsRestoreSnapshotDoneResponse)
       IsRestoreSnapshotDoneResponseOrBuilder {
     // Use IsRestoreSnapshotDoneResponse.newBuilder() to construct.
-    private IsRestoreSnapshotDoneResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsRestoreSnapshotDoneResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsRestoreSnapshotDoneResponse() {
@@ -45399,18 +45399,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsRestoreSnapshotDoneResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -45433,22 +45433,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -45481,7 +45481,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, done_);
@@ -45495,7 +45495,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, done_);
       }
       size += unknownFields.getSerializedSize();
@@ -45533,7 +45533,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasDone()) {
         hash = (37 * hash) + DONE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDone());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -45542,61 +45542,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -45614,7 +45614,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -45622,15 +45622,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsRestoreSnapshotDoneResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsRestoreSnapshotDoneResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -45643,12 +45643,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -45659,7 +45659,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_descriptor;
       }
@@ -45693,29 +45693,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse)other);
         } else {
@@ -45739,13 +45739,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -45789,12 +45789,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -45812,22 +45812,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsRestoreSnapshotDoneResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsRestoreSnapshotDoneResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -45839,7 +45839,7 @@ public final class MasterProtos {
 
   public interface GetSchemaAlterStatusRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetSchemaAlterStatusRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -45858,29 +45858,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetSchemaAlterStatusRequest}
    */
   public  static final class GetSchemaAlterStatusRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetSchemaAlterStatusRequest)
       GetSchemaAlterStatusRequestOrBuilder {
     // Use GetSchemaAlterStatusRequest.newBuilder() to construct.
-    private GetSchemaAlterStatusRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetSchemaAlterStatusRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetSchemaAlterStatusRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetSchemaAlterStatusRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -45911,22 +45911,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -45973,7 +45973,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -45987,7 +45987,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       size += unknownFields.getSerializedSize();
@@ -46033,61 +46033,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -46105,7 +46105,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -46113,15 +46113,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetSchemaAlterStatusRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetSchemaAlterStatusRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -46134,12 +46134,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -46155,7 +46155,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusRequest_descriptor;
       }
@@ -46193,29 +46193,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest)other);
         } else {
@@ -46245,13 +46245,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -46264,7 +46264,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -46367,11 +46367,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -46381,12 +46381,12 @@ public final class MasterProtos {
         return tableNameBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -46404,22 +46404,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetSchemaAlterStatusRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetSchemaAlterStatusRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -46431,7 +46431,7 @@ public final class MasterProtos {
 
   public interface GetSchemaAlterStatusResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetSchemaAlterStatusResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint32 yet_to_update_regions = 1;
@@ -46455,11 +46455,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetSchemaAlterStatusResponse}
    */
   public  static final class GetSchemaAlterStatusResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetSchemaAlterStatusResponse)
       GetSchemaAlterStatusResponseOrBuilder {
     // Use GetSchemaAlterStatusResponse.newBuilder() to construct.
-    private GetSchemaAlterStatusResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetSchemaAlterStatusResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetSchemaAlterStatusResponse() {
@@ -46468,18 +46468,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetSchemaAlterStatusResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -46507,22 +46507,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -46570,7 +46570,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, yetToUpdateRegions_);
@@ -46587,11 +46587,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, yetToUpdateRegions_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, totalRegions_);
       }
       size += unknownFields.getSerializedSize();
@@ -46646,61 +46646,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -46718,7 +46718,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -46726,15 +46726,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetSchemaAlterStatusResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetSchemaAlterStatusResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -46747,12 +46747,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -46765,7 +46765,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetSchemaAlterStatusResponse_descriptor;
       }
@@ -46803,29 +46803,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse)other);
         } else {
@@ -46852,13 +46852,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -46934,12 +46934,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -46957,22 +46957,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetSchemaAlterStatusResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetSchemaAlterStatusResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -46984,7 +46984,7 @@ public final class MasterProtos {
 
   public interface GetTableDescriptorsRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetTableDescriptorsRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.TableName table_names = 1;
@@ -47021,7 +47021,7 @@ public final class MasterProtos {
     /**
      * optional string regex = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRegexBytes();
 
     /**
@@ -47044,18 +47044,18 @@ public final class MasterProtos {
     /**
      * optional string namespace = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.GetTableDescriptorsRequest}
    */
   public  static final class GetTableDescriptorsRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetTableDescriptorsRequest)
       GetTableDescriptorsRequestOrBuilder {
     // Use GetTableDescriptorsRequest.newBuilder() to construct.
-    private GetTableDescriptorsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetTableDescriptorsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetTableDescriptorsRequest() {
@@ -47066,18 +47066,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetTableDescriptorsRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -47103,7 +47103,7 @@ public final class MasterProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               regex_ = bs;
               break;
@@ -47114,17 +47114,17 @@ public final class MasterProtos {
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               namespace_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -47134,12 +47134,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -47198,8 +47198,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           regex_ = s;
@@ -47210,17 +47210,17 @@ public final class MasterProtos {
     /**
      * optional string regex = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRegexBytes() {
       java.lang.Object ref = regex_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         regex_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -47255,8 +47255,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespace_ = s;
@@ -47267,17 +47267,17 @@ public final class MasterProtos {
     /**
      * optional string namespace = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceBytes() {
       java.lang.Object ref = namespace_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespace_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -47297,19 +47297,19 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < tableNames_.size(); i++) {
         output.writeMessage(1, tableNames_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, regex_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, regex_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(3, includeSysTables_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, namespace_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, namespace_);
       }
       unknownFields.writeTo(output);
     }
@@ -47320,18 +47320,18 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < tableNames_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, tableNames_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, regex_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, regex_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, includeSysTables_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, namespace_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, namespace_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -47388,7 +47388,7 @@ public final class MasterProtos {
       }
       if (hasIncludeSysTables()) {
         hash = (37 * hash) + INCLUDE_SYS_TABLES_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIncludeSysTables());
       }
       if (hasNamespace()) {
@@ -47401,61 +47401,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -47473,7 +47473,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -47481,15 +47481,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetTableDescriptorsRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetTableDescriptorsRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -47502,12 +47502,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNamesFieldBuilder();
         }
@@ -47529,7 +47529,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsRequest_descriptor;
       }
@@ -47580,29 +47580,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest)other);
         } else {
@@ -47632,7 +47632,7 @@ public final class MasterProtos {
               tableNames_ = other.tableNames_;
               bitField0_ = (bitField0_ & ~0x00000001);
               tableNamesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableNamesFieldBuilder() : null;
             } else {
               tableNamesBuilder_.addAllMessages(other.tableNames_);
@@ -47667,13 +47667,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -47694,7 +47694,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNamesBuilder_;
 
       /**
@@ -47826,7 +47826,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (tableNamesBuilder_ == null) {
           ensureTableNamesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, tableNames_);
           onChanged();
         } else {
@@ -47910,11 +47910,11 @@ public final class MasterProtos {
            getTableNamesBuilderList() {
         return getTableNamesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNamesFieldBuilder() {
         if (tableNamesBuilder_ == null) {
-          tableNamesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          tableNamesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   tableNames_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -47938,8 +47938,8 @@ public final class MasterProtos {
       public java.lang.String getRegex() {
         java.lang.Object ref = regex_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             regex_ = s;
@@ -47952,17 +47952,17 @@ public final class MasterProtos {
       /**
        * optional string regex = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getRegexBytes() {
         java.lang.Object ref = regex_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           regex_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -47991,7 +47991,7 @@ public final class MasterProtos {
        * optional string regex = 2;
        */
       public Builder setRegexBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -48046,8 +48046,8 @@ public final class MasterProtos {
       public java.lang.String getNamespace() {
         java.lang.Object ref = namespace_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespace_ = s;
@@ -48060,17 +48060,17 @@ public final class MasterProtos {
       /**
        * optional string namespace = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceBytes() {
         java.lang.Object ref = namespace_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespace_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -48099,7 +48099,7 @@ public final class MasterProtos {
        * optional string namespace = 4;
        */
       public Builder setNamespaceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -48109,12 +48109,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -48132,22 +48132,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetTableDescriptorsRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetTableDescriptorsRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -48159,7 +48159,7 @@ public final class MasterProtos {
 
   public interface GetTableDescriptorsResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetTableDescriptorsResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.TableSchema table_schema = 1;
@@ -48189,11 +48189,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetTableDescriptorsResponse}
    */
   public  static final class GetTableDescriptorsResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetTableDescriptorsResponse)
       GetTableDescriptorsResponseOrBuilder {
     // Use GetTableDescriptorsResponse.newBuilder() to construct.
-    private GetTableDescriptorsResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetTableDescriptorsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetTableDescriptorsResponse() {
@@ -48201,18 +48201,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetTableDescriptorsResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -48239,10 +48239,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -48252,12 +48252,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -48315,7 +48315,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < tableSchema_.size(); i++) {
         output.writeMessage(1, tableSchema_.get(i));
@@ -48329,7 +48329,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < tableSchema_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, tableSchema_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -48372,61 +48372,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -48444,7 +48444,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -48452,15 +48452,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetTableDescriptorsResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetTableDescriptorsResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -48473,12 +48473,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableSchemaFieldBuilder();
         }
@@ -48494,7 +48494,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableDescriptorsResponse_descriptor;
       }
@@ -48531,29 +48531,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse)other);
         } else {
@@ -48583,7 +48583,7 @@ public final class MasterProtos {
               tableSchema_ = other.tableSchema_;
               bitField0_ = (bitField0_ & ~0x00000001);
               tableSchemaBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableSchemaFieldBuilder() : null;
             } else {
               tableSchemaBuilder_.addAllMessages(other.tableSchema_);
@@ -48605,13 +48605,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -48632,7 +48632,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
 
       /**
@@ -48764,7 +48764,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (tableSchemaBuilder_ == null) {
           ensureTableSchemaIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, tableSchema_);
           onChanged();
         } else {
@@ -48848,11 +48848,11 @@ public final class MasterProtos {
            getTableSchemaBuilderList() {
         return getTableSchemaFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   tableSchema_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -48863,12 +48863,12 @@ public final class MasterProtos {
         return tableSchemaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -48886,22 +48886,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetTableDescriptorsResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetTableDescriptorsResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -48913,7 +48913,7 @@ public final class MasterProtos {
 
   public interface GetTableNamesRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetTableNamesRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional string regex = 1;
@@ -48926,7 +48926,7 @@ public final class MasterProtos {
     /**
      * optional string regex = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRegexBytes();
 
     /**
@@ -48949,18 +48949,18 @@ public final class MasterProtos {
     /**
      * optional string namespace = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.GetTableNamesRequest}
    */
   public  static final class GetTableNamesRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetTableNamesRequest)
       GetTableNamesRequestOrBuilder {
     // Use GetTableNamesRequest.newBuilder() to construct.
-    private GetTableNamesRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetTableNamesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetTableNamesRequest() {
@@ -48970,18 +48970,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetTableNamesRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -48998,7 +48998,7 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               regex_ = bs;
               break;
@@ -49009,29 +49009,29 @@ public final class MasterProtos {
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               namespace_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -49055,8 +49055,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           regex_ = s;
@@ -49067,17 +49067,17 @@ public final class MasterProtos {
     /**
      * optional string regex = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRegexBytes() {
       java.lang.Object ref = regex_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         regex_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -49112,8 +49112,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespace_ = s;
@@ -49124,17 +49124,17 @@ public final class MasterProtos {
     /**
      * optional string namespace = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceBytes() {
       java.lang.Object ref = namespace_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespace_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -49148,16 +49148,16 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, regex_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, regex_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(2, includeSysTables_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, namespace_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, namespace_);
       }
       unknownFields.writeTo(output);
     }
@@ -49168,14 +49168,14 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, regex_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, regex_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, includeSysTables_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, namespace_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, namespace_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -49226,7 +49226,7 @@ public final class MasterProtos {
       }
       if (hasIncludeSysTables()) {
         hash = (37 * hash) + INCLUDE_SYS_TABLES_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIncludeSysTables());
       }
       if (hasNamespace()) {
@@ -49239,61 +49239,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -49311,7 +49311,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -49319,15 +49319,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetTableNamesRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetTableNamesRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -49340,12 +49340,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -49360,7 +49360,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesRequest_descriptor;
       }
@@ -49402,29 +49402,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest)other);
         } else {
@@ -49458,13 +49458,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -49489,8 +49489,8 @@ public final class MasterProtos {
       public java.lang.String getRegex() {
         java.lang.Object ref = regex_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             regex_ = s;
@@ -49503,17 +49503,17 @@ public final class MasterProtos {
       /**
        * optional string regex = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getRegexBytes() {
         java.lang.Object ref = regex_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           regex_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -49542,7 +49542,7 @@ public final class MasterProtos {
        * optional string regex = 1;
        */
       public Builder setRegexBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -49597,8 +49597,8 @@ public final class MasterProtos {
       public java.lang.String getNamespace() {
         java.lang.Object ref = namespace_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespace_ = s;
@@ -49611,17 +49611,17 @@ public final class MasterProtos {
       /**
        * optional string namespace = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceBytes() {
         java.lang.Object ref = namespace_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespace_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -49650,7 +49650,7 @@ public final class MasterProtos {
        * optional string namespace = 3;
        */
       public Builder setNamespaceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -49660,12 +49660,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -49683,22 +49683,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetTableNamesRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetTableNamesRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -49710,7 +49710,7 @@ public final class MasterProtos {
 
   public interface GetTableNamesResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetTableNamesResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.TableName table_names = 1;
@@ -49740,11 +49740,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetTableNamesResponse}
    */
   public  static final class GetTableNamesResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetTableNamesResponse)
       GetTableNamesResponseOrBuilder {
     // Use GetTableNamesResponse.newBuilder() to construct.
-    private GetTableNamesResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetTableNamesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetTableNamesResponse() {
@@ -49752,18 +49752,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetTableNamesResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -49790,10 +49790,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -49803,12 +49803,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -49866,7 +49866,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < tableNames_.size(); i++) {
         output.writeMessage(1, tableNames_.get(i));
@@ -49880,7 +49880,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < tableNames_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, tableNames_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -49923,61 +49923,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -49995,7 +49995,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -50003,15 +50003,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetTableNamesResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetTableNamesResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -50024,12 +50024,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNamesFieldBuilder();
         }
@@ -50045,7 +50045,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableNamesResponse_descriptor;
       }
@@ -50082,29 +50082,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse)other);
         } else {
@@ -50134,7 +50134,7 @@ public final class MasterProtos {
               tableNames_ = other.tableNames_;
               bitField0_ = (bitField0_ & ~0x00000001);
               tableNamesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableNamesFieldBuilder() : null;
             } else {
               tableNamesBuilder_.addAllMessages(other.tableNames_);
@@ -50156,13 +50156,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -50183,7 +50183,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNamesBuilder_;
 
       /**
@@ -50315,7 +50315,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (tableNamesBuilder_ == null) {
           ensureTableNamesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, tableNames_);
           onChanged();
         } else {
@@ -50399,11 +50399,11 @@ public final class MasterProtos {
            getTableNamesBuilderList() {
         return getTableNamesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNamesFieldBuilder() {
         if (tableNamesBuilder_ == null) {
-          tableNamesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          tableNamesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   tableNames_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -50414,12 +50414,12 @@ public final class MasterProtos {
         return tableNamesBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -50437,22 +50437,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetTableNamesResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetTableNamesResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -50464,7 +50464,7 @@ public final class MasterProtos {
 
   public interface GetTableStateRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetTableStateRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -50483,29 +50483,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetTableStateRequest}
    */
   public  static final class GetTableStateRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetTableStateRequest)
       GetTableStateRequestOrBuilder {
     // Use GetTableStateRequest.newBuilder() to construct.
-    private GetTableStateRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetTableStateRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetTableStateRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetTableStateRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -50536,22 +50536,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -50598,7 +50598,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -50612,7 +50612,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       size += unknownFields.getSerializedSize();
@@ -50658,61 +50658,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -50730,7 +50730,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -50738,15 +50738,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetTableStateRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetTableStateRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -50759,12 +50759,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -50780,7 +50780,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateRequest_descriptor;
       }
@@ -50818,29 +50818,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)other);
         } else {
@@ -50870,13 +50870,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -50889,7 +50889,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -50992,11 +50992,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -51006,12 +51006,12 @@ public final class MasterProtos {
         return tableNameBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -51029,22 +51029,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetTableStateRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetTableStateRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -51056,7 +51056,7 @@ public final class MasterProtos {
 
   public interface GetTableStateResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetTableStateResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableState table_state = 1;
@@ -51075,29 +51075,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetTableStateResponse}
    */
   public  static final class GetTableStateResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetTableStateResponse)
       GetTableStateResponseOrBuilder {
     // Use GetTableStateResponse.newBuilder() to construct.
-    private GetTableStateResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetTableStateResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetTableStateResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetTableStateResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -51128,22 +51128,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -51190,7 +51190,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableState());
@@ -51204,7 +51204,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableState());
       }
       size += unknownFields.getSerializedSize();
@@ -51250,61 +51250,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -51322,7 +51322,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -51330,15 +51330,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetTableStateResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetTableStateResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -51351,12 +51351,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableStateFieldBuilder();
         }
@@ -51372,7 +51372,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetTableStateResponse_descriptor;
       }
@@ -51410,29 +51410,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse)other);
         } else {
@@ -51462,13 +51462,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -51481,7 +51481,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState tableState_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableStateOrBuilder> tableStateBuilder_;
       /**
        * required .hbase.pb.TableState table_state = 1;
@@ -51584,11 +51584,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableState table_state = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableStateOrBuilder> 
           getTableStateFieldBuilder() {
         if (tableStateBuilder_ == null) {
-          tableStateBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableStateBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableStateOrBuilder>(
                   getTableState(),
                   getParentForChildren(),
@@ -51598,12 +51598,12 @@ public final class MasterProtos {
         return tableStateBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -51621,22 +51621,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetTableStateResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetTableStateResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -51648,34 +51648,34 @@ public final class MasterProtos {
 
   public interface GetClusterStatusRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetClusterStatusRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.GetClusterStatusRequest}
    */
   public  static final class GetClusterStatusRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetClusterStatusRequest)
       GetClusterStatusRequestOrBuilder {
     // Use GetClusterStatusRequest.newBuilder() to construct.
-    private GetClusterStatusRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetClusterStatusRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetClusterStatusRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetClusterStatusRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -51693,22 +51693,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -51725,7 +51725,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -51769,61 +51769,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -51841,7 +51841,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -51849,15 +51849,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetClusterStatusRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetClusterStatusRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -51870,12 +51870,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -51884,7 +51884,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusRequest_descriptor;
       }
@@ -51911,29 +51911,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest)other);
         } else {
@@ -51954,13 +51954,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -51971,12 +51971,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -51994,22 +51994,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetClusterStatusRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetClusterStatusRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -52021,7 +52021,7 @@ public final class MasterProtos {
 
   public interface GetClusterStatusResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetClusterStatusResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ClusterStatus cluster_status = 1;
@@ -52040,29 +52040,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetClusterStatusResponse}
    */
   public  static final class GetClusterStatusResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetClusterStatusResponse)
       GetClusterStatusResponseOrBuilder {
     // Use GetClusterStatusResponse.newBuilder() to construct.
-    private GetClusterStatusResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetClusterStatusResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetClusterStatusResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetClusterStatusResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -52093,22 +52093,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -52155,7 +52155,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getClusterStatus());
@@ -52169,7 +52169,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getClusterStatus());
       }
       size += unknownFields.getSerializedSize();
@@ -52215,61 +52215,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -52287,7 +52287,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -52295,15 +52295,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetClusterStatusResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetClusterStatusResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -52316,12 +52316,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getClusterStatusFieldBuilder();
         }
@@ -52337,7 +52337,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetClusterStatusResponse_descriptor;
       }
@@ -52375,29 +52375,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse)other);
         } else {
@@ -52427,13 +52427,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -52446,7 +52446,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus clusterStatus_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatusOrBuilder> clusterStatusBuilder_;
       /**
        * required .hbase.pb.ClusterStatus cluster_status = 1;
@@ -52549,11 +52549,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.ClusterStatus cluster_status = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatusOrBuilder> 
           getClusterStatusFieldBuilder() {
         if (clusterStatusBuilder_ == null) {
-          clusterStatusBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          clusterStatusBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ClusterStatusOrBuilder>(
                   getClusterStatus(),
                   getParentForChildren(),
@@ -52563,12 +52563,12 @@ public final class MasterProtos {
         return clusterStatusBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -52586,22 +52586,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetClusterStatusResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetClusterStatusResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -52613,34 +52613,34 @@ public final class MasterProtos {
 
   public interface IsMasterRunningRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsMasterRunningRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.IsMasterRunningRequest}
    */
   public  static final class IsMasterRunningRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsMasterRunningRequest)
       IsMasterRunningRequestOrBuilder {
     // Use IsMasterRunningRequest.newBuilder() to construct.
-    private IsMasterRunningRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsMasterRunningRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsMasterRunningRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsMasterRunningRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -52658,22 +52658,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -52690,7 +52690,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -52734,61 +52734,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -52806,7 +52806,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -52814,15 +52814,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsMasterRunningRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsMasterRunningRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -52835,12 +52835,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -52849,7 +52849,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningRequest_descriptor;
       }
@@ -52876,29 +52876,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest)other);
         } else {
@@ -52919,13 +52919,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -52936,12 +52936,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -52959,22 +52959,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsMasterRunningRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsMasterRunningRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -52986,7 +52986,7 @@ public final class MasterProtos {
 
   public interface IsMasterRunningResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsMasterRunningResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool is_master_running = 1;
@@ -53001,11 +53001,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsMasterRunningResponse}
    */
   public  static final class IsMasterRunningResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsMasterRunningResponse)
       IsMasterRunningResponseOrBuilder {
     // Use IsMasterRunningResponse.newBuilder() to construct.
-    private IsMasterRunningResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsMasterRunningResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsMasterRunningResponse() {
@@ -53013,18 +53013,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsMasterRunningResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -53047,22 +53047,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -53099,7 +53099,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, isMasterRunning_);
@@ -53113,7 +53113,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, isMasterRunning_);
       }
       size += unknownFields.getSerializedSize();
@@ -53151,7 +53151,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasIsMasterRunning()) {
         hash = (37 * hash) + IS_MASTER_RUNNING_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIsMasterRunning());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -53160,61 +53160,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -53232,7 +53232,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -53240,15 +53240,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsMasterRunningResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsMasterRunningResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -53261,12 +53261,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -53277,7 +53277,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsMasterRunningResponse_descriptor;
       }
@@ -53311,29 +53311,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse)other);
         } else {
@@ -53360,13 +53360,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -53410,12 +53410,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -53433,22 +53433,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsMasterRunningResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsMasterRunningResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -53460,7 +53460,7 @@ public final class MasterProtos {
 
   public interface ExecProcedureRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ExecProcedureRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ProcedureDescription procedure = 1;
@@ -53479,29 +53479,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ExecProcedureRequest}
    */
   public  static final class ExecProcedureRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ExecProcedureRequest)
       ExecProcedureRequestOrBuilder {
     // Use ExecProcedureRequest.newBuilder() to construct.
-    private ExecProcedureRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ExecProcedureRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ExecProcedureRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ExecProcedureRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -53532,22 +53532,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -53594,7 +53594,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getProcedure());
@@ -53608,7 +53608,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getProcedure());
       }
       size += unknownFields.getSerializedSize();
@@ -53654,61 +53654,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -53726,7 +53726,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -53734,15 +53734,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ExecProcedureRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ExecProcedureRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -53755,12 +53755,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getProcedureFieldBuilder();
         }
@@ -53776,7 +53776,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureRequest_descriptor;
       }
@@ -53814,29 +53814,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)other);
         } else {
@@ -53866,13 +53866,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -53885,7 +53885,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription procedure_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> procedureBuilder_;
       /**
        * required .hbase.pb.ProcedureDescription procedure = 1;
@@ -53988,11 +53988,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.ProcedureDescription procedure = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> 
           getProcedureFieldBuilder() {
         if (procedureBuilder_ == null) {
-          procedureBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          procedureBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder>(
                   getProcedure(),
                   getParentForChildren(),
@@ -54002,12 +54002,12 @@ public final class MasterProtos {
         return procedureBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -54025,22 +54025,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ExecProcedureRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ExecProcedureRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -54052,7 +54052,7 @@ public final class MasterProtos {
 
   public interface ExecProcedureResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ExecProcedureResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional int64 expected_timeout = 1;
@@ -54070,37 +54070,37 @@ public final class MasterProtos {
     /**
      * optional bytes return_data = 2;
      */
-    com.google.protobuf.ByteString getReturnData();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getReturnData();
   }
   /**
    * Protobuf type {@code hbase.pb.ExecProcedureResponse}
    */
   public  static final class ExecProcedureResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ExecProcedureResponse)
       ExecProcedureResponseOrBuilder {
     // Use ExecProcedureResponse.newBuilder() to construct.
-    private ExecProcedureResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ExecProcedureResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ExecProcedureResponse() {
       expectedTimeout_ = 0L;
-      returnData_ = com.google.protobuf.ByteString.EMPTY;
+      returnData_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ExecProcedureResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -54128,22 +54128,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -54167,7 +54167,7 @@ public final class MasterProtos {
     }
 
     public static final int RETURN_DATA_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString returnData_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString returnData_;
     /**
      * optional bytes return_data = 2;
      */
@@ -54177,7 +54177,7 @@ public final class MasterProtos {
     /**
      * optional bytes return_data = 2;
      */
-    public com.google.protobuf.ByteString getReturnData() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getReturnData() {
       return returnData_;
     }
 
@@ -54191,7 +54191,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, expectedTimeout_);
@@ -54208,11 +54208,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, expectedTimeout_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, returnData_);
       }
       size += unknownFields.getSerializedSize();
@@ -54255,7 +54255,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasExpectedTimeout()) {
         hash = (37 * hash) + EXPECTED_TIMEOUT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getExpectedTimeout());
       }
       if (hasReturnData()) {
@@ -54268,61 +54268,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -54340,7 +54340,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -54348,15 +54348,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ExecProcedureResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ExecProcedureResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -54369,12 +54369,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -54382,12 +54382,12 @@ public final class MasterProtos {
         super.clear();
         expectedTimeout_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000001);
-        returnData_ = com.google.protobuf.ByteString.EMPTY;
+        returnData_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ExecProcedureResponse_descriptor;
       }
@@ -54425,29 +54425,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse)other);
         } else {
@@ -54474,13 +54474,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -54524,7 +54524,7 @@ public final class MasterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString returnData_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString returnData_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes return_data = 2;
        */
@@ -54534,13 +54534,13 @@ public final class MasterProtos {
       /**
        * optional bytes return_data = 2;
        */
-      public com.google.protobuf.ByteString getReturnData() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getReturnData() {
         return returnData_;
       }
       /**
        * optional bytes return_data = 2;
        */
-      public Builder setReturnData(com.google.protobuf.ByteString value) {
+      public Builder setReturnData(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -54559,12 +54559,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -54582,22 +54582,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ExecProcedureResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ExecProcedureResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -54609,7 +54609,7 @@ public final class MasterProtos {
 
   public interface IsProcedureDoneRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsProcedureDoneRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.ProcedureDescription procedure = 1;
@@ -54628,29 +54628,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsProcedureDoneRequest}
    */
   public  static final class IsProcedureDoneRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsProcedureDoneRequest)
       IsProcedureDoneRequestOrBuilder {
     // Use IsProcedureDoneRequest.newBuilder() to construct.
-    private IsProcedureDoneRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsProcedureDoneRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsProcedureDoneRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsProcedureDoneRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -54681,22 +54681,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -54741,7 +54741,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getProcedure());
@@ -54755,7 +54755,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getProcedure());
       }
       size += unknownFields.getSerializedSize();
@@ -54801,61 +54801,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -54873,7 +54873,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -54881,15 +54881,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsProcedureDoneRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsProcedureDoneRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -54902,12 +54902,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getProcedureFieldBuilder();
         }
@@ -54923,7 +54923,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneRequest_descriptor;
       }
@@ -54961,29 +54961,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest)other);
         } else {
@@ -55012,13 +55012,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -55031,7 +55031,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription procedure_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> procedureBuilder_;
       /**
        * optional .hbase.pb.ProcedureDescription procedure = 1;
@@ -55134,11 +55134,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.ProcedureDescription procedure = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> 
           getProcedureFieldBuilder() {
         if (procedureBuilder_ == null) {
-          procedureBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          procedureBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder>(
                   getProcedure(),
                   getParentForChildren(),
@@ -55148,12 +55148,12 @@ public final class MasterProtos {
         return procedureBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -55171,22 +55171,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsProcedureDoneRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsProcedureDoneRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -55198,7 +55198,7 @@ public final class MasterProtos {
 
   public interface IsProcedureDoneResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.IsProcedureDoneResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool done = 1 [default = false];
@@ -55226,11 +55226,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.IsProcedureDoneResponse}
    */
   public  static final class IsProcedureDoneResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.IsProcedureDoneResponse)
       IsProcedureDoneResponseOrBuilder {
     // Use IsProcedureDoneResponse.newBuilder() to construct.
-    private IsProcedureDoneResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private IsProcedureDoneResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private IsProcedureDoneResponse() {
@@ -55238,18 +55238,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private IsProcedureDoneResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -55285,22 +55285,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -55360,7 +55360,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, done_);
@@ -55377,11 +55377,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, done_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getSnapshot());
       }
       size += unknownFields.getSerializedSize();
@@ -55424,7 +55424,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasDone()) {
         hash = (37 * hash) + DONE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDone());
       }
       if (hasSnapshot()) {
@@ -55437,61 +55437,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -55509,7 +55509,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -55517,15 +55517,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.IsProcedureDoneResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.IsProcedureDoneResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -55538,12 +55538,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getSnapshotFieldBuilder();
         }
@@ -55561,7 +55561,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsProcedureDoneResponse_descriptor;
       }
@@ -55603,29 +55603,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse)other);
         } else {
@@ -55657,13 +55657,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -55708,7 +55708,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription snapshot_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> snapshotBuilder_;
       /**
        * optional .hbase.pb.ProcedureDescription snapshot = 2;
@@ -55811,11 +55811,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.ProcedureDescription snapshot = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
-          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder>(
                   getSnapshot(),
                   getParentForChildren(),
@@ -55825,12 +55825,12 @@ public final class MasterProtos {
         return snapshotBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -55848,22 +55848,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public IsProcedureDoneResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new IsProcedureDoneResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -55875,7 +55875,7 @@ public final class MasterProtos {
 
   public interface GetProcedureResultRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetProcedureResultRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 proc_id = 1;
@@ -55890,11 +55890,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetProcedureResultRequest}
    */
   public  static final class GetProcedureResultRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetProcedureResultRequest)
       GetProcedureResultRequestOrBuilder {
     // Use GetProcedureResultRequest.newBuilder() to construct.
-    private GetProcedureResultRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetProcedureResultRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetProcedureResultRequest() {
@@ -55902,18 +55902,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetProcedureResultRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -55936,22 +55936,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -55988,7 +55988,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -56002,7 +56002,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       size += unknownFields.getSerializedSize();
@@ -56040,7 +56040,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -56049,61 +56049,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -56121,7 +56121,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -56129,15 +56129,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetProcedureResultRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetProcedureResultRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -56150,12 +56150,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -56166,7 +56166,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultRequest_descriptor;
       }
@@ -56200,29 +56200,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)other);
         } else {
@@ -56249,13 +56249,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -56299,12 +56299,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -56322,22 +56322,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetProcedureResultRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetProcedureResultRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -56349,7 +56349,7 @@ public final class MasterProtos {
 
   public interface GetProcedureResultResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetProcedureResultResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.GetProcedureResultResponse.State state = 1;
@@ -56385,7 +56385,7 @@ public final class MasterProtos {
     /**
      * optional bytes result = 4;
      */
-    com.google.protobuf.ByteString getResult();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getResult();
 
     /**
      * optional .hbase.pb.ForeignExceptionMessage exception = 5;
@@ -56404,33 +56404,33 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.GetProcedureResultResponse}
    */
   public  static final class GetProcedureResultResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetProcedureResultResponse)
       GetProcedureResultResponseOrBuilder {
     // Use GetProcedureResultResponse.newBuilder() to construct.
-    private GetProcedureResultResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetProcedureResultResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetProcedureResultResponse() {
       state_ = 0;
       startTime_ = 0L;
       lastUpdate_ = 0L;
-      result_ = com.google.protobuf.ByteString.EMPTY;
+      result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetProcedureResultResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -56487,22 +56487,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -56513,7 +56513,7 @@ public final class MasterProtos {
      * Protobuf enum {@code hbase.pb.GetProcedureResultResponse.State}
      */
     public enum State
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * NOT_FOUND = 0;
        */
@@ -56563,27 +56563,27 @@ public final class MasterProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           State> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public State findValueByNumber(int number) {
                 return State.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDescriptor().getEnumTypes().get(0);
       }
@@ -56591,7 +56591,7 @@ public final class MasterProtos {
       private static final State[] VALUES = values();
 
       public static State valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -56656,7 +56656,7 @@ public final class MasterProtos {
     }
 
     public static final int RESULT_FIELD_NUMBER = 4;
-    private com.google.protobuf.ByteString result_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString result_;
     /**
      * optional bytes result = 4;
      */
@@ -56666,7 +56666,7 @@ public final class MasterProtos {
     /**
      * optional bytes result = 4;
      */
-    public com.google.protobuf.ByteString getResult() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getResult() {
       return result_;
     }
 
@@ -56705,7 +56705,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, state_);
@@ -56731,23 +56731,23 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, state_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, startTime_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, lastUpdate_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(4, result_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getException());
       }
       size += unknownFields.getSerializedSize();
@@ -56808,12 +56808,12 @@ public final class MasterProtos {
       }
       if (hasStartTime()) {
         hash = (37 * hash) + START_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getStartTime());
       }
       if (hasLastUpdate()) {
         hash = (37 * hash) + LAST_UPDATE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastUpdate());
       }
       if (hasResult()) {
@@ -56830,61 +56830,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -56902,7 +56902,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -56910,15 +56910,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.GetProcedureResultResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetProcedureResultResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -56931,12 +56931,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getExceptionFieldBuilder();
         }
@@ -56949,7 +56949,7 @@ public final class MasterProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         lastUpdate_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
-        result_ = com.google.protobuf.ByteString.EMPTY;
+        result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
         if (exceptionBuilder_ == null) {
           exception_ = null;
@@ -56960,7 +56960,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetProcedureResultResponse_descriptor;
       }
@@ -57014,29 +57014,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse)other);
         } else {
@@ -57075,13 +57075,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -57193,7 +57193,7 @@ public final class MasterProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString result_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes result = 4;
        */
@@ -57203,13 +57203,13 @@ public final class MasterProtos {
       /**
        * optional bytes result = 4;
        */
-      public com.google.protobuf.ByteString getResult() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getResult() {
         return result_;
       }
       /**
        * optional bytes result = 4;
        */
-      public Builder setResult(com.google.protobuf.ByteString value) {
+      public Builder setResult(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -57229,7 +57229,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage exception_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder> exceptionBuilder_;
       /**
        * optional .hbase.pb.ForeignExceptionMessage exception = 5;
@@ -57332,11 +57332,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.ForeignExceptionMessage exception = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder> 
           getExceptionFieldBuilder() {
         if (exceptionBuilder_ == null) {
-          exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          exceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder>(
                   getException(),
                   getParentForChildren(),
@@ -57346,12 +57346,12 @@ public final class MasterProtos {
         return exceptionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -57369,22 +57369,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetProcedureResultResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetProcedureResultResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -57396,7 +57396,7 @@ public final class MasterProtos {
 
   public interface AbortProcedureRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AbortProcedureRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint64 proc_id = 1;
@@ -57420,11 +57420,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.AbortProcedureRequest}
    */
   public  static final class AbortProcedureRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AbortProcedureRequest)
       AbortProcedureRequestOrBuilder {
     // Use AbortProcedureRequest.newBuilder() to construct.
-    private AbortProcedureRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AbortProcedureRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AbortProcedureRequest() {
@@ -57433,18 +57433,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AbortProcedureRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -57472,22 +57472,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -57539,7 +57539,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, procId_);
@@ -57556,11 +57556,11 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, procId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, mayInterruptIfRunning_);
       }
       size += unknownFields.getSerializedSize();
@@ -57603,12 +57603,12 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       if (hasMayInterruptIfRunning()) {
         hash = (37 * hash) + MAYINTERRUPTIFRUNNING_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getMayInterruptIfRunning());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -57617,61 +57617,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -57689,7 +57689,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -57697,15 +57697,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.AbortProcedureRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AbortProcedureRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -57718,12 +57718,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -57736,7 +57736,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureRequest_descriptor;
       }
@@ -57774,29 +57774,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)other);
         } else {
@@ -57826,13 +57826,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -57908,12 +57908,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -57931,22 +57931,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AbortProcedureRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AbortProcedureRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -57958,7 +57958,7 @@ public final class MasterProtos {
 
   public interface AbortProcedureResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.AbortProcedureResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool is_procedure_aborted = 1;
@@ -57973,11 +57973,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.AbortProcedureResponse}
    */
   public  static final class AbortProcedureResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.AbortProcedureResponse)
       AbortProcedureResponseOrBuilder {
     // Use AbortProcedureResponse.newBuilder() to construct.
-    private AbortProcedureResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private AbortProcedureResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private AbortProcedureResponse() {
@@ -57985,18 +57985,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private AbortProcedureResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -58019,22 +58019,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -58071,7 +58071,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, isProcedureAborted_);
@@ -58085,7 +58085,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, isProcedureAborted_);
       }
       size += unknownFields.getSerializedSize();
@@ -58123,7 +58123,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasIsProcedureAborted()) {
         hash = (37 * hash) + IS_PROCEDURE_ABORTED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIsProcedureAborted());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -58132,61 +58132,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -58204,7 +58204,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -58212,15 +58212,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.AbortProcedureResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.AbortProcedureResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -58233,12 +58233,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -58249,7 +58249,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_AbortProcedureResponse_descriptor;
       }
@@ -58283,29 +58283,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse)other);
         } else {
@@ -58332,13 +58332,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -58382,12 +58382,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -58405,22 +58405,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public AbortProcedureResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new AbortProcedureResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -58432,34 +58432,34 @@ public final class MasterProtos {
 
   public interface ListProceduresRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListProceduresRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.ListProceduresRequest}
    */
   public  static final class ListProceduresRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListProceduresRequest)
       ListProceduresRequestOrBuilder {
     // Use ListProceduresRequest.newBuilder() to construct.
-    private ListProceduresRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListProceduresRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListProceduresRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListProceduresRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -58477,22 +58477,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -58509,7 +58509,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -58553,61 +58553,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -58625,7 +58625,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -58633,15 +58633,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListProceduresRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListProceduresRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -58654,12 +58654,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -58668,7 +58668,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresRequest_descriptor;
       }
@@ -58695,29 +58695,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest)other);
         } else {
@@ -58738,13 +58738,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -58755,12 +58755,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -58778,22 +58778,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListProceduresRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListProceduresRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -58805,7 +58805,7 @@ public final class MasterProtos {
 
   public interface ListProceduresResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ListProceduresResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.Procedure procedure = 1;
@@ -58835,11 +58835,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.ListProceduresResponse}
    */
   public  static final class ListProceduresResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ListProceduresResponse)
       ListProceduresResponseOrBuilder {
     // Use ListProceduresResponse.newBuilder() to construct.
-    private ListProceduresResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ListProceduresResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ListProceduresResponse() {
@@ -58847,18 +58847,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ListProceduresResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -58885,10 +58885,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -58898,12 +58898,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -58961,7 +58961,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < procedure_.size(); i++) {
         output.writeMessage(1, procedure_.get(i));
@@ -58975,7 +58975,7 @@ public final class MasterProtos {
 
       size = 0;
       for (int i = 0; i < procedure_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, procedure_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -59018,61 +59018,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -59090,7 +59090,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -59098,15 +59098,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.ListProceduresResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ListProceduresResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -59119,12 +59119,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getProcedureFieldBuilder();
         }
@@ -59140,7 +59140,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListProceduresResponse_descriptor;
       }
@@ -59177,29 +59177,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse)other);
         } else {
@@ -59229,7 +59229,7 @@ public final class MasterProtos {
               procedure_ = other.procedure_;
               bitField0_ = (bitField0_ & ~0x00000001);
               procedureBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getProcedureFieldBuilder() : null;
             } else {
               procedureBuilder_.addAllMessages(other.procedure_);
@@ -59251,13 +59251,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -59278,7 +59278,7 @@ public final class MasterProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> procedureBuilder_;
 
       /**
@@ -59410,7 +59410,7 @@ public final class MasterProtos {
           java.lang.Iterable values) {
         if (procedureBuilder_ == null) {
           ensureProcedureIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, procedure_);
           onChanged();
         } else {
@@ -59494,11 +59494,11 @@ public final class MasterProtos {
            getProcedureBuilderList() {
         return getProcedureFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> 
           getProcedureFieldBuilder() {
         if (procedureBuilder_ == null) {
-          procedureBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          procedureBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder>(
                   procedure_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -59509,12 +59509,12 @@ public final class MasterProtos {
         return procedureBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -59532,22 +59532,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ListProceduresResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ListProceduresResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -59559,7 +59559,7 @@ public final class MasterProtos {
 
   public interface SetQuotaRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetQuotaRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional string user_name = 1;
@@ -59572,7 +59572,7 @@ public final class MasterProtos {
     /**
      * optional string user_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUserNameBytes();
 
     /**
@@ -59586,7 +59586,7 @@ public final class MasterProtos {
     /**
      * optional string user_group = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUserGroupBytes();
 
     /**
@@ -59600,7 +59600,7 @@ public final class MasterProtos {
     /**
      * optional string namespace = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceBytes();
 
     /**
@@ -59651,11 +59651,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SetQuotaRequest}
    */
   public  static final class SetQuotaRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetQuotaRequest)
       SetQuotaRequestOrBuilder {
     // Use SetQuotaRequest.newBuilder() to construct.
-    private SetQuotaRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetQuotaRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetQuotaRequest() {
@@ -59667,18 +59667,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetQuotaRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -59695,19 +59695,19 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               userName_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               userGroup_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               namespace_ = bs;
               break;
@@ -59750,22 +59750,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -59789,8 +59789,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           userName_ = s;
@@ -59801,17 +59801,17 @@ public final class MasterProtos {
     /**
      * optional string user_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUserNameBytes() {
       java.lang.Object ref = userName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         userName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -59831,8 +59831,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           userGroup_ = s;
@@ -59843,17 +59843,17 @@ public final class MasterProtos {
     /**
      * optional string user_group = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUserGroupBytes() {
       java.lang.Object ref = userGroup_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         userGroup_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -59873,8 +59873,8 @@ public final class MasterProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           namespace_ = s;
@@ -59885,17 +59885,17 @@ public final class MasterProtos {
     /**
      * optional string namespace = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNamespaceBytes() {
       java.lang.Object ref = namespace_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         namespace_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -59993,16 +59993,16 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, userName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, userName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, userGroup_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, userGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, namespace_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, namespace_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeMessage(4, getTableName());
@@ -60025,28 +60025,28 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, userName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, userName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, userGroup_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, userGroup_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, namespace_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, namespace_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getTableName());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, removeAll_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(6, bypassGlobals_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(7, getThrottle());
       }
       size += unknownFields.getSerializedSize();
@@ -60130,12 +60130,12 @@ public final class MasterProtos {
       }
       if (hasRemoveAll()) {
         hash = (37 * hash) + REMOVE_ALL_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getRemoveAll());
       }
       if (hasBypassGlobals()) {
         hash = (37 * hash) + BYPASS_GLOBALS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getBypassGlobals());
       }
       if (hasThrottle()) {
@@ -60148,61 +60148,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -60220,7 +60220,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -60228,15 +60228,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetQuotaRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetQuotaRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -60249,12 +60249,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getThrottleFieldBuilder();
@@ -60287,7 +60287,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaRequest_descriptor;
       }
@@ -60353,29 +60353,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)other);
         } else {
@@ -60433,13 +60433,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -60464,8 +60464,8 @@ public final class MasterProtos {
       public java.lang.String getUserName() {
         java.lang.Object ref = userName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             userName_ = s;
@@ -60478,17 +60478,17 @@ public final class MasterProtos {
       /**
        * optional string user_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getUserNameBytes() {
         java.lang.Object ref = userName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           userName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -60517,7 +60517,7 @@ public final class MasterProtos {
        * optional string user_name = 1;
        */
       public Builder setUserNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -60540,8 +60540,8 @@ public final class MasterProtos {
       public java.lang.String getUserGroup() {
         java.lang.Object ref = userGroup_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             userGroup_ = s;
@@ -60554,17 +60554,17 @@ public final class MasterProtos {
       /**
        * optional string user_group = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getUserGroupBytes() {
         java.lang.Object ref = userGroup_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           userGroup_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -60593,7 +60593,7 @@ public final class MasterProtos {
        * optional string user_group = 2;
        */
       public Builder setUserGroupBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -60616,8 +60616,8 @@ public final class MasterProtos {
       public java.lang.String getNamespace() {
         java.lang.Object ref = namespace_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             namespace_ = s;
@@ -60630,17 +60630,17 @@ public final class MasterProtos {
       /**
        * optional string namespace = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNamespaceBytes() {
         java.lang.Object ref = namespace_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespace_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -60669,7 +60669,7 @@ public final class MasterProtos {
        * optional string namespace = 3;
        */
       public Builder setNamespaceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -60680,7 +60680,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * optional .hbase.pb.TableName table_name = 4;
@@ -60783,11 +60783,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.TableName table_name = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -60862,7 +60862,7 @@ public final class MasterProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest throttle_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequestOrBuilder> throttleBuilder_;
       /**
        * optional .hbase.pb.ThrottleRequest throttle = 7;
@@ -60965,11 +60965,11 @@ public final class MasterProtos {
       /**
        * optional .hbase.pb.ThrottleRequest throttle = 7;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequestOrBuilder> 
           getThrottleFieldBuilder() {
         if (throttleBuilder_ == null) {
-          throttleBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          throttleBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequestOrBuilder>(
                   getThrottle(),
                   getParentForChildren(),
@@ -60979,12 +60979,12 @@ public final class MasterProtos {
         return throttleBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -61002,22 +61002,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetQuotaRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetQuotaRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -61029,34 +61029,34 @@ public final class MasterProtos {
 
   public interface SetQuotaResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetQuotaResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.SetQuotaResponse}
    */
   public  static final class SetQuotaResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SetQuotaResponse)
       SetQuotaResponseOrBuilder {
     // Use SetQuotaResponse.newBuilder() to construct.
-    private SetQuotaResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SetQuotaResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SetQuotaResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SetQuotaResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -61074,22 +61074,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -61106,7 +61106,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -61150,61 +61150,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -61222,7 +61222,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -61230,15 +61230,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SetQuotaResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SetQuotaResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -61251,12 +61251,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -61265,7 +61265,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetQuotaResponse_descriptor;
       }
@@ -61292,29 +61292,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse)other);
         } else {
@@ -61335,13 +61335,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -61352,12 +61352,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -61375,22 +61375,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SetQuotaResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SetQuotaResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -61402,7 +61402,7 @@ public final class MasterProtos {
 
   public interface MajorCompactionTimestampRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MajorCompactionTimestampRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -61421,29 +61421,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.MajorCompactionTimestampRequest}
    */
   public  static final class MajorCompactionTimestampRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MajorCompactionTimestampRequest)
       MajorCompactionTimestampRequestOrBuilder {
     // Use MajorCompactionTimestampRequest.newBuilder() to construct.
-    private MajorCompactionTimestampRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MajorCompactionTimestampRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MajorCompactionTimestampRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MajorCompactionTimestampRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -61474,22 +61474,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -61536,7 +61536,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -61550,7 +61550,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       size += unknownFields.getSerializedSize();
@@ -61596,61 +61596,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -61668,7 +61668,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -61676,15 +61676,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.MajorCompactionTimestampRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MajorCompactionTimestampRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -61697,12 +61697,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -61718,7 +61718,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampRequest_descriptor;
       }
@@ -61756,29 +61756,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)other);
         } else {
@@ -61808,13 +61808,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -61827,7 +61827,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -61930,11 +61930,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -61944,12 +61944,12 @@ public final class MasterProtos {
         return tableNameBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -61967,22 +61967,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MajorCompactionTimestampRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MajorCompactionTimestampRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -61994,7 +61994,7 @@ public final class MasterProtos {
 
   public interface MajorCompactionTimestampForRegionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MajorCompactionTimestampForRegionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionSpecifier region = 1;
@@ -62013,29 +62013,29 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.MajorCompactionTimestampForRegionRequest}
    */
   public  static final class MajorCompactionTimestampForRegionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MajorCompactionTimestampForRegionRequest)
       MajorCompactionTimestampForRegionRequestOrBuilder {
     // Use MajorCompactionTimestampForRegionRequest.newBuilder() to construct.
-    private MajorCompactionTimestampForRegionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MajorCompactionTimestampForRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MajorCompactionTimestampForRegionRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MajorCompactionTimestampForRegionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -62066,22 +62066,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -62128,7 +62128,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getRegion());
@@ -62142,7 +62142,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getRegion());
       }
       size += unknownFields.getSerializedSize();
@@ -62188,61 +62188,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -62260,7 +62260,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -62268,15 +62268,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.MajorCompactionTimestampForRegionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MajorCompactionTimestampForRegionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -62289,12 +62289,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionFieldBuilder();
         }
@@ -62310,7 +62310,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_descriptor;
       }
@@ -62348,29 +62348,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)other);
         } else {
@@ -62400,13 +62400,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -62419,7 +62419,7 @@ public final class MasterProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
@@ -62522,11 +62522,11 @@ public final class MasterProtos {
       /**
        * required .hbase.pb.RegionSpecifier region = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
           getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          regionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
                   getRegion(),
                   getParentForChildren(),
@@ -62536,12 +62536,12 @@ public final class MasterProtos {
         return regionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -62559,22 +62559,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MajorCompactionTimestampForRegionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MajorCompactionTimestampForRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -62586,7 +62586,7 @@ public final class MasterProtos {
 
   public interface MajorCompactionTimestampResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MajorCompactionTimestampResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int64 compaction_timestamp = 1;
@@ -62601,11 +62601,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.MajorCompactionTimestampResponse}
    */
   public  static final class MajorCompactionTimestampResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MajorCompactionTimestampResponse)
       MajorCompactionTimestampResponseOrBuilder {
     // Use MajorCompactionTimestampResponse.newBuilder() to construct.
-    private MajorCompactionTimestampResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MajorCompactionTimestampResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MajorCompactionTimestampResponse() {
@@ -62613,18 +62613,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MajorCompactionTimestampResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -62647,22 +62647,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -62699,7 +62699,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, compactionTimestamp_);
@@ -62713,7 +62713,7 @@ public final class MasterProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, compactionTimestamp_);
       }
       size += unknownFields.getSerializedSize();
@@ -62751,7 +62751,7 @@ public final class MasterProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasCompactionTimestamp()) {
         hash = (37 * hash) + COMPACTION_TIMESTAMP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getCompactionTimestamp());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -62760,61 +62760,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -62832,7 +62832,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -62840,15 +62840,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.MajorCompactionTimestampResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MajorCompactionTimestampResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -62861,12 +62861,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -62877,7 +62877,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MajorCompactionTimestampResponse_descriptor;
       }
@@ -62911,29 +62911,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse)other);
         } else {
@@ -62960,13 +62960,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -63010,12 +63010,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -63033,22 +63033,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MajorCompactionTimestampResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MajorCompactionTimestampResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -63060,34 +63060,34 @@ public final class MasterProtos {
 
   public interface SecurityCapabilitiesRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SecurityCapabilitiesRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.SecurityCapabilitiesRequest}
    */
   public  static final class SecurityCapabilitiesRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SecurityCapabilitiesRequest)
       SecurityCapabilitiesRequestOrBuilder {
     // Use SecurityCapabilitiesRequest.newBuilder() to construct.
-    private SecurityCapabilitiesRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SecurityCapabilitiesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SecurityCapabilitiesRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SecurityCapabilitiesRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -63105,22 +63105,22 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -63137,7 +63137,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -63181,61 +63181,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -63253,7 +63253,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -63261,15 +63261,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SecurityCapabilitiesRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SecurityCapabilitiesRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -63282,12 +63282,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -63296,7 +63296,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesRequest_descriptor;
       }
@@ -63323,29 +63323,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)other);
         } else {
@@ -63366,13 +63366,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -63383,12 +63383,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -63406,22 +63406,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SecurityCapabilitiesRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SecurityCapabilitiesRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -63433,7 +63433,7 @@ public final class MasterProtos {
 
   public interface SecurityCapabilitiesResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SecurityCapabilitiesResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;
@@ -63452,11 +63452,11 @@ public final class MasterProtos {
    * Protobuf type {@code hbase.pb.SecurityCapabilitiesResponse}
    */
   public  static final class SecurityCapabilitiesResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SecurityCapabilitiesResponse)
       SecurityCapabilitiesResponseOrBuilder {
     // Use SecurityCapabilitiesResponse.newBuilder() to construct.
-    private SecurityCapabilitiesResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SecurityCapabilitiesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SecurityCapabilitiesResponse() {
@@ -63464,18 +63464,18 @@ public final class MasterProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SecurityCapabilitiesResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -63526,10 +63526,10 @@ public final class MasterProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -63539,12 +63539,12 @@ public final class MasterProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -63555,7 +63555,7 @@ public final class MasterProtos {
      * Protobuf enum {@code hbase.pb.SecurityCapabilitiesResponse.Capability}
      */
     public enum Capability
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * SIMPLE_AUTHENTICATION = 0;
        */
@@ -63623,27 +63623,27 @@ public final class MasterProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Capability> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Capability findValueByNumber(int number) {
                 return Capability.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDescriptor().getEnumTypes().get(0);
       }
@@ -63651,7 +63651,7 @@ public final class MasterProtos {
       private static final Capability[] VALUES = values();
 
       public static Capability valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -63670,9 +63670,9 @@ public final class MasterProtos {
 
     public static final int CAPABILITIES_FIELD_NUMBER = 1;
     private java.util.List capabilities_;
-    private static final com.google.protobuf.Internal.ListAdapter.Converter<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter.Converter<
         java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability> capabilities_converter_ =
-            new com.google.protobuf.Internal.ListAdapter.Converter<
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter.Converter<
                 java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability>() {
               public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability convert(java.lang.Integer from) {
                 org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability result = org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability.valueOf(from);
@@ -63683,7 +63683,7 @@ public final class MasterProtos {
      * repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;
      */
     public java.util.List getCapabilitiesList() {
-      return new com.google.protobuf.Internal.ListAdapter<
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter<
           java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability>(capabilities_, capabilities_converter_);
     }
     /**
@@ -63709,7 +63709,7 @@ public final class MasterProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < capabilities_.size(); i++) {
         output.writeEnum(1, capabilities_.get(i));
@@ -63725,7 +63725,7 @@ public final class MasterProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < capabilities_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeEnumSizeNoTag(capabilities_.get(i));
         }
         size += dataSize;
@@ -63770,61 +63770,61 @@ public final class MasterProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -63842,7 +63842,7 @@ public final class MasterProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -63850,15 +63850,15 @@ public final class MasterProtos {
      * Protobuf type {@code hbase.pb.SecurityCapabilitiesResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SecurityCapabilitiesResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -63871,12 +63871,12 @@ public final class MasterProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -63887,7 +63887,7 @@ public final class MasterProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
       }
@@ -63920,29 +63920,29 @@ public final class MasterProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse)other);
         } else {
@@ -63973,13 +63973,13 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -64003,7 +64003,7 @@ public final class MasterProtos {
        * repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;
        */
       public java.util.List getCapabilitiesList() {
-        return new com.google.protobuf.Internal.ListAdapter<
+        return new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ListAdapter<
             java.lang.Integer, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability>(capabilities_, capabilities_converter_);
       }
       /**
@@ -64065,12 +64065,12 @@ public final class MasterProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -64088,22 +64088,22 @@ public final class MasterProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SecurityCapabilitiesResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SecurityCapabilitiesResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -64117,7 +64117,7 @@ public final class MasterProtos {
    * Protobuf service {@code hbase.pb.MasterService}
    */
   public static abstract class MasterService
-      implements com.google.protobuf.Service {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.Service {
     protected MasterService() {}
 
     public interface Interface {
@@ -64129,9 +64129,9 @@ public final class MasterProtos {
        * rpc GetSchemaAlterStatus(.hbase.pb.GetSchemaAlterStatusRequest) returns (.hbase.pb.GetSchemaAlterStatusResponse);
        */
       public abstract void getSchemaAlterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64141,9 +64141,9 @@ public final class MasterProtos {
        * rpc GetTableDescriptors(.hbase.pb.GetTableDescriptorsRequest) returns (.hbase.pb.GetTableDescriptorsResponse);
        */
       public abstract void getTableDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64153,9 +64153,9 @@ public final class MasterProtos {
        * rpc GetTableNames(.hbase.pb.GetTableNamesRequest) returns (.hbase.pb.GetTableNamesResponse);
        */
       public abstract void getTableNames(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64165,9 +64165,9 @@ public final class MasterProtos {
        * rpc GetClusterStatus(.hbase.pb.GetClusterStatusRequest) returns (.hbase.pb.GetClusterStatusResponse);
        */
       public abstract void getClusterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64177,9 +64177,9 @@ public final class MasterProtos {
        * rpc IsMasterRunning(.hbase.pb.IsMasterRunningRequest) returns (.hbase.pb.IsMasterRunningResponse);
        */
       public abstract void isMasterRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64189,9 +64189,9 @@ public final class MasterProtos {
        * rpc AddColumn(.hbase.pb.AddColumnRequest) returns (.hbase.pb.AddColumnResponse);
        */
       public abstract void addColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64201,9 +64201,9 @@ public final class MasterProtos {
        * rpc DeleteColumn(.hbase.pb.DeleteColumnRequest) returns (.hbase.pb.DeleteColumnResponse);
        */
       public abstract void deleteColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64213,9 +64213,9 @@ public final class MasterProtos {
        * rpc ModifyColumn(.hbase.pb.ModifyColumnRequest) returns (.hbase.pb.ModifyColumnResponse);
        */
       public abstract void modifyColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64225,9 +64225,9 @@ public final class MasterProtos {
        * rpc MoveRegion(.hbase.pb.MoveRegionRequest) returns (.hbase.pb.MoveRegionResponse);
        */
       public abstract void moveRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64237,9 +64237,9 @@ public final class MasterProtos {
        * rpc DispatchMergingRegions(.hbase.pb.DispatchMergingRegionsRequest) returns (.hbase.pb.DispatchMergingRegionsResponse);
        */
       public abstract void dispatchMergingRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64249,9 +64249,9 @@ public final class MasterProtos {
        * rpc AssignRegion(.hbase.pb.AssignRegionRequest) returns (.hbase.pb.AssignRegionResponse);
        */
       public abstract void assignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64265,9 +64265,9 @@ public final class MasterProtos {
        * rpc UnassignRegion(.hbase.pb.UnassignRegionRequest) returns (.hbase.pb.UnassignRegionResponse);
        */
       public abstract void unassignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64281,9 +64281,9 @@ public final class MasterProtos {
        * rpc OfflineRegion(.hbase.pb.OfflineRegionRequest) returns (.hbase.pb.OfflineRegionResponse);
        */
       public abstract void offlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64293,9 +64293,9 @@ public final class MasterProtos {
        * rpc DeleteTable(.hbase.pb.DeleteTableRequest) returns (.hbase.pb.DeleteTableResponse);
        */
       public abstract void deleteTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64305,9 +64305,9 @@ public final class MasterProtos {
        * rpc truncateTable(.hbase.pb.TruncateTableRequest) returns (.hbase.pb.TruncateTableResponse);
        */
       public abstract void truncateTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64317,9 +64317,9 @@ public final class MasterProtos {
        * rpc EnableTable(.hbase.pb.EnableTableRequest) returns (.hbase.pb.EnableTableResponse);
        */
       public abstract void enableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64329,9 +64329,9 @@ public final class MasterProtos {
        * rpc DisableTable(.hbase.pb.DisableTableRequest) returns (.hbase.pb.DisableTableResponse);
        */
       public abstract void disableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64341,9 +64341,9 @@ public final class MasterProtos {
        * rpc ModifyTable(.hbase.pb.ModifyTableRequest) returns (.hbase.pb.ModifyTableResponse);
        */
       public abstract void modifyTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64353,9 +64353,9 @@ public final class MasterProtos {
        * rpc CreateTable(.hbase.pb.CreateTableRequest) returns (.hbase.pb.CreateTableResponse);
        */
       public abstract void createTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64365,9 +64365,9 @@ public final class MasterProtos {
        * rpc Shutdown(.hbase.pb.ShutdownRequest) returns (.hbase.pb.ShutdownResponse);
        */
       public abstract void shutdown(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64377,9 +64377,9 @@ public final class MasterProtos {
        * rpc StopMaster(.hbase.pb.StopMasterRequest) returns (.hbase.pb.StopMasterResponse);
        */
       public abstract void stopMaster(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64390,9 +64390,9 @@ public final class MasterProtos {
        * rpc IsMasterInMaintenanceMode(.hbase.pb.IsInMaintenanceModeRequest) returns (.hbase.pb.IsInMaintenanceModeResponse);
        */
       public abstract void isMasterInMaintenanceMode(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64405,9 +64405,9 @@ public final class MasterProtos {
        * rpc Balance(.hbase.pb.BalanceRequest) returns (.hbase.pb.BalanceResponse);
        */
       public abstract void balance(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64419,9 +64419,9 @@ public final class MasterProtos {
        * rpc SetBalancerRunning(.hbase.pb.SetBalancerRunningRequest) returns (.hbase.pb.SetBalancerRunningResponse);
        */
       public abstract void setBalancerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64432,9 +64432,9 @@ public final class MasterProtos {
        * rpc IsBalancerEnabled(.hbase.pb.IsBalancerEnabledRequest) returns (.hbase.pb.IsBalancerEnabledResponse);
        */
       public abstract void isBalancerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64446,9 +64446,9 @@ public final class MasterProtos {
        * rpc SetSplitOrMergeEnabled(.hbase.pb.SetSplitOrMergeEnabledRequest) returns (.hbase.pb.SetSplitOrMergeEnabledResponse);
        */
       public abstract void setSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64459,9 +64459,9 @@ public final class MasterProtos {
        * rpc IsSplitOrMergeEnabled(.hbase.pb.IsSplitOrMergeEnabledRequest) returns (.hbase.pb.IsSplitOrMergeEnabledResponse);
        */
       public abstract void isSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64472,9 +64472,9 @@ public final class MasterProtos {
        * rpc Normalize(.hbase.pb.NormalizeRequest) returns (.hbase.pb.NormalizeResponse);
        */
       public abstract void normalize(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64485,9 +64485,9 @@ public final class MasterProtos {
        * rpc SetNormalizerRunning(.hbase.pb.SetNormalizerRunningRequest) returns (.hbase.pb.SetNormalizerRunningResponse);
        */
       public abstract void setNormalizerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64498,9 +64498,9 @@ public final class MasterProtos {
        * rpc IsNormalizerEnabled(.hbase.pb.IsNormalizerEnabledRequest) returns (.hbase.pb.IsNormalizerEnabledResponse);
        */
       public abstract void isNormalizerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64510,9 +64510,9 @@ public final class MasterProtos {
        * rpc RunCatalogScan(.hbase.pb.RunCatalogScanRequest) returns (.hbase.pb.RunCatalogScanResponse);
        */
       public abstract void runCatalogScan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64523,9 +64523,9 @@ public final class MasterProtos {
        * rpc EnableCatalogJanitor(.hbase.pb.EnableCatalogJanitorRequest) returns (.hbase.pb.EnableCatalogJanitorResponse);
        */
       public abstract void enableCatalogJanitor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64536,9 +64536,9 @@ public final class MasterProtos {
        * rpc IsCatalogJanitorEnabled(.hbase.pb.IsCatalogJanitorEnabledRequest) returns (.hbase.pb.IsCatalogJanitorEnabledResponse);
        */
       public abstract void isCatalogJanitorEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64549,9 +64549,9 @@ public final class MasterProtos {
        * rpc ExecMasterService(.hbase.pb.CoprocessorServiceRequest) returns (.hbase.pb.CoprocessorServiceResponse);
        */
       public abstract void execMasterService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64562,9 +64562,9 @@ public final class MasterProtos {
        * rpc Snapshot(.hbase.pb.SnapshotRequest) returns (.hbase.pb.SnapshotResponse);
        */
       public abstract void snapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64576,9 +64576,9 @@ public final class MasterProtos {
        * rpc GetCompletedSnapshots(.hbase.pb.GetCompletedSnapshotsRequest) returns (.hbase.pb.GetCompletedSnapshotsResponse);
        */
       public abstract void getCompletedSnapshots(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64589,9 +64589,9 @@ public final class MasterProtos {
        * rpc DeleteSnapshot(.hbase.pb.DeleteSnapshotRequest) returns (.hbase.pb.DeleteSnapshotResponse);
        */
       public abstract void deleteSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64602,9 +64602,9 @@ public final class MasterProtos {
        * rpc IsSnapshotDone(.hbase.pb.IsSnapshotDoneRequest) returns (.hbase.pb.IsSnapshotDoneResponse);
        */
       public abstract void isSnapshotDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64615,9 +64615,9 @@ public final class MasterProtos {
        * rpc RestoreSnapshot(.hbase.pb.RestoreSnapshotRequest) returns (.hbase.pb.RestoreSnapshotResponse);
        */
       public abstract void restoreSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64628,9 +64628,9 @@ public final class MasterProtos {
        * rpc ExecProcedure(.hbase.pb.ExecProcedureRequest) returns (.hbase.pb.ExecProcedureResponse);
        */
       public abstract void execProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64641,9 +64641,9 @@ public final class MasterProtos {
        * rpc ExecProcedureWithRet(.hbase.pb.ExecProcedureRequest) returns (.hbase.pb.ExecProcedureResponse);
        */
       public abstract void execProcedureWithRet(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64654,9 +64654,9 @@ public final class MasterProtos {
        * rpc IsProcedureDone(.hbase.pb.IsProcedureDoneRequest) returns (.hbase.pb.IsProcedureDoneResponse);
        */
       public abstract void isProcedureDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64666,9 +64666,9 @@ public final class MasterProtos {
        * rpc ModifyNamespace(.hbase.pb.ModifyNamespaceRequest) returns (.hbase.pb.ModifyNamespaceResponse);
        */
       public abstract void modifyNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64678,9 +64678,9 @@ public final class MasterProtos {
        * rpc CreateNamespace(.hbase.pb.CreateNamespaceRequest) returns (.hbase.pb.CreateNamespaceResponse);
        */
       public abstract void createNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64690,9 +64690,9 @@ public final class MasterProtos {
        * rpc DeleteNamespace(.hbase.pb.DeleteNamespaceRequest) returns (.hbase.pb.DeleteNamespaceResponse);
        */
       public abstract void deleteNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64702,9 +64702,9 @@ public final class MasterProtos {
        * rpc GetNamespaceDescriptor(.hbase.pb.GetNamespaceDescriptorRequest) returns (.hbase.pb.GetNamespaceDescriptorResponse);
        */
       public abstract void getNamespaceDescriptor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64714,9 +64714,9 @@ public final class MasterProtos {
        * rpc ListNamespaceDescriptors(.hbase.pb.ListNamespaceDescriptorsRequest) returns (.hbase.pb.ListNamespaceDescriptorsResponse);
        */
       public abstract void listNamespaceDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64726,9 +64726,9 @@ public final class MasterProtos {
        * rpc ListTableDescriptorsByNamespace(.hbase.pb.ListTableDescriptorsByNamespaceRequest) returns (.hbase.pb.ListTableDescriptorsByNamespaceResponse);
        */
       public abstract void listTableDescriptorsByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64738,9 +64738,9 @@ public final class MasterProtos {
        * rpc ListTableNamesByNamespace(.hbase.pb.ListTableNamesByNamespaceRequest) returns (.hbase.pb.ListTableNamesByNamespaceResponse);
        */
       public abstract void listTableNamesByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64750,9 +64750,9 @@ public final class MasterProtos {
        * rpc GetTableState(.hbase.pb.GetTableStateRequest) returns (.hbase.pb.GetTableStateResponse);
        */
       public abstract void getTableState(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64762,9 +64762,9 @@ public final class MasterProtos {
        * rpc SetQuota(.hbase.pb.SetQuotaRequest) returns (.hbase.pb.SetQuotaResponse);
        */
       public abstract void setQuota(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64774,9 +64774,9 @@ public final class MasterProtos {
        * rpc getLastMajorCompactionTimestamp(.hbase.pb.MajorCompactionTimestampRequest) returns (.hbase.pb.MajorCompactionTimestampResponse);
        */
       public abstract void getLastMajorCompactionTimestamp(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64786,17 +64786,17 @@ public final class MasterProtos {
        * rpc getLastMajorCompactionTimestampForRegion(.hbase.pb.MajorCompactionTimestampForRegionRequest) returns (.hbase.pb.MajorCompactionTimestampResponse);
        */
       public abstract void getLastMajorCompactionTimestampForRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * rpc getProcedureResult(.hbase.pb.GetProcedureResultRequest) returns (.hbase.pb.GetProcedureResultResponse);
        */
       public abstract void getProcedureResult(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64806,9 +64806,9 @@ public final class MasterProtos {
        * rpc getSecurityCapabilities(.hbase.pb.SecurityCapabilitiesRequest) returns (.hbase.pb.SecurityCapabilitiesResponse);
        */
       public abstract void getSecurityCapabilities(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64818,9 +64818,9 @@ public final class MasterProtos {
        * rpc AbortProcedure(.hbase.pb.AbortProcedureRequest) returns (.hbase.pb.AbortProcedureResponse);
        */
       public abstract void abortProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -64830,487 +64830,487 @@ public final class MasterProtos {
        * rpc ListProcedures(.hbase.pb.ListProceduresRequest) returns (.hbase.pb.ListProceduresResponse);
        */
       public abstract void listProcedures(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     }
 
-    public static com.google.protobuf.Service newReflectiveService(
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
         final Interface impl) {
       return new MasterService() {
         @java.lang.Override
         public  void getSchemaAlterStatus(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getSchemaAlterStatus(controller, request, done);
         }
 
         @java.lang.Override
         public  void getTableDescriptors(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getTableDescriptors(controller, request, done);
         }
 
         @java.lang.Override
         public  void getTableNames(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getTableNames(controller, request, done);
         }
 
         @java.lang.Override
         public  void getClusterStatus(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getClusterStatus(controller, request, done);
         }
 
         @java.lang.Override
         public  void isMasterRunning(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isMasterRunning(controller, request, done);
         }
 
         @java.lang.Override
         public  void addColumn(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.addColumn(controller, request, done);
         }
 
         @java.lang.Override
         public  void deleteColumn(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.deleteColumn(controller, request, done);
         }
 
         @java.lang.Override
         public  void modifyColumn(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.modifyColumn(controller, request, done);
         }
 
         @java.lang.Override
         public  void moveRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.moveRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void dispatchMergingRegions(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.dispatchMergingRegions(controller, request, done);
         }
 
         @java.lang.Override
         public  void assignRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.assignRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void unassignRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.unassignRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void offlineRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.offlineRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void deleteTable(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.deleteTable(controller, request, done);
         }
 
         @java.lang.Override
         public  void truncateTable(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.truncateTable(controller, request, done);
         }
 
         @java.lang.Override
         public  void enableTable(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.enableTable(controller, request, done);
         }
 
         @java.lang.Override
         public  void disableTable(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.disableTable(controller, request, done);
         }
 
         @java.lang.Override
         public  void modifyTable(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.modifyTable(controller, request, done);
         }
 
         @java.lang.Override
         public  void createTable(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.createTable(controller, request, done);
         }
 
         @java.lang.Override
         public  void shutdown(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.shutdown(controller, request, done);
         }
 
         @java.lang.Override
         public  void stopMaster(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.stopMaster(controller, request, done);
         }
 
         @java.lang.Override
         public  void isMasterInMaintenanceMode(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isMasterInMaintenanceMode(controller, request, done);
         }
 
         @java.lang.Override
         public  void balance(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.balance(controller, request, done);
         }
 
         @java.lang.Override
         public  void setBalancerRunning(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.setBalancerRunning(controller, request, done);
         }
 
         @java.lang.Override
         public  void isBalancerEnabled(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isBalancerEnabled(controller, request, done);
         }
 
         @java.lang.Override
         public  void setSplitOrMergeEnabled(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.setSplitOrMergeEnabled(controller, request, done);
         }
 
         @java.lang.Override
         public  void isSplitOrMergeEnabled(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isSplitOrMergeEnabled(controller, request, done);
         }
 
         @java.lang.Override
         public  void normalize(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.normalize(controller, request, done);
         }
 
         @java.lang.Override
         public  void setNormalizerRunning(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.setNormalizerRunning(controller, request, done);
         }
 
         @java.lang.Override
         public  void isNormalizerEnabled(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isNormalizerEnabled(controller, request, done);
         }
 
         @java.lang.Override
         public  void runCatalogScan(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.runCatalogScan(controller, request, done);
         }
 
         @java.lang.Override
         public  void enableCatalogJanitor(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.enableCatalogJanitor(controller, request, done);
         }
 
         @java.lang.Override
         public  void isCatalogJanitorEnabled(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isCatalogJanitorEnabled(controller, request, done);
         }
 
         @java.lang.Override
         public  void execMasterService(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.execMasterService(controller, request, done);
         }
 
         @java.lang.Override
         public  void snapshot(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.snapshot(controller, request, done);
         }
 
         @java.lang.Override
         public  void getCompletedSnapshots(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getCompletedSnapshots(controller, request, done);
         }
 
         @java.lang.Override
         public  void deleteSnapshot(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.deleteSnapshot(controller, request, done);
         }
 
         @java.lang.Override
         public  void isSnapshotDone(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isSnapshotDone(controller, request, done);
         }
 
         @java.lang.Override
         public  void restoreSnapshot(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.restoreSnapshot(controller, request, done);
         }
 
         @java.lang.Override
         public  void execProcedure(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.execProcedure(controller, request, done);
         }
 
         @java.lang.Override
         public  void execProcedureWithRet(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.execProcedureWithRet(controller, request, done);
         }
 
         @java.lang.Override
         public  void isProcedureDone(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.isProcedureDone(controller, request, done);
         }
 
         @java.lang.Override
         public  void modifyNamespace(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.modifyNamespace(controller, request, done);
         }
 
         @java.lang.Override
         public  void createNamespace(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.createNamespace(controller, request, done);
         }
 
         @java.lang.Override
         public  void deleteNamespace(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.deleteNamespace(controller, request, done);
         }
 
         @java.lang.Override
         public  void getNamespaceDescriptor(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getNamespaceDescriptor(controller, request, done);
         }
 
         @java.lang.Override
         public  void listNamespaceDescriptors(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.listNamespaceDescriptors(controller, request, done);
         }
 
         @java.lang.Override
         public  void listTableDescriptorsByNamespace(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.listTableDescriptorsByNamespace(controller, request, done);
         }
 
         @java.lang.Override
         public  void listTableNamesByNamespace(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.listTableNamesByNamespace(controller, request, done);
         }
 
         @java.lang.Override
         public  void getTableState(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getTableState(controller, request, done);
         }
 
         @java.lang.Override
         public  void setQuota(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.setQuota(controller, request, done);
         }
 
         @java.lang.Override
         public  void getLastMajorCompactionTimestamp(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getLastMajorCompactionTimestamp(controller, request, done);
         }
 
         @java.lang.Override
         public  void getLastMajorCompactionTimestampForRegion(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getLastMajorCompactionTimestampForRegion(controller, request, done);
         }
 
         @java.lang.Override
         public  void getProcedureResult(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getProcedureResult(controller, request, done);
         }
 
         @java.lang.Override
         public  void getSecurityCapabilities(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getSecurityCapabilities(controller, request, done);
         }
 
         @java.lang.Override
         public  void abortProcedure(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.abortProcedure(controller, request, done);
         }
 
         @java.lang.Override
         public  void listProcedures(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.listProcedures(controller, request, done);
         }
 
       };
     }
 
-    public static com.google.protobuf.BlockingService
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
         newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService() {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
             getDescriptorForType() {
           return getDescriptor();
         }
 
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message callBlockingMethod(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.callBlockingMethod() given method descriptor for " +
@@ -65436,9 +65436,9 @@ public final class MasterProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getRequestPrototype() given method " +
@@ -65564,9 +65564,9 @@ public final class MasterProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getResponsePrototype() given method " +
@@ -65703,9 +65703,9 @@ public final class MasterProtos {
      * rpc GetSchemaAlterStatus(.hbase.pb.GetSchemaAlterStatusRequest) returns (.hbase.pb.GetSchemaAlterStatusResponse);
      */
     public abstract void getSchemaAlterStatus(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65715,9 +65715,9 @@ public final class MasterProtos {
      * rpc GetTableDescriptors(.hbase.pb.GetTableDescriptorsRequest) returns (.hbase.pb.GetTableDescriptorsResponse);
      */
     public abstract void getTableDescriptors(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65727,9 +65727,9 @@ public final class MasterProtos {
      * rpc GetTableNames(.hbase.pb.GetTableNamesRequest) returns (.hbase.pb.GetTableNamesResponse);
      */
     public abstract void getTableNames(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65739,9 +65739,9 @@ public final class MasterProtos {
      * rpc GetClusterStatus(.hbase.pb.GetClusterStatusRequest) returns (.hbase.pb.GetClusterStatusResponse);
      */
     public abstract void getClusterStatus(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65751,9 +65751,9 @@ public final class MasterProtos {
      * rpc IsMasterRunning(.hbase.pb.IsMasterRunningRequest) returns (.hbase.pb.IsMasterRunningResponse);
      */
     public abstract void isMasterRunning(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65763,9 +65763,9 @@ public final class MasterProtos {
      * rpc AddColumn(.hbase.pb.AddColumnRequest) returns (.hbase.pb.AddColumnResponse);
      */
     public abstract void addColumn(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65775,9 +65775,9 @@ public final class MasterProtos {
      * rpc DeleteColumn(.hbase.pb.DeleteColumnRequest) returns (.hbase.pb.DeleteColumnResponse);
      */
     public abstract void deleteColumn(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65787,9 +65787,9 @@ public final class MasterProtos {
      * rpc ModifyColumn(.hbase.pb.ModifyColumnRequest) returns (.hbase.pb.ModifyColumnResponse);
      */
     public abstract void modifyColumn(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65799,9 +65799,9 @@ public final class MasterProtos {
      * rpc MoveRegion(.hbase.pb.MoveRegionRequest) returns (.hbase.pb.MoveRegionResponse);
      */
     public abstract void moveRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65811,9 +65811,9 @@ public final class MasterProtos {
      * rpc DispatchMergingRegions(.hbase.pb.DispatchMergingRegionsRequest) returns (.hbase.pb.DispatchMergingRegionsResponse);
      */
     public abstract void dispatchMergingRegions(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65823,9 +65823,9 @@ public final class MasterProtos {
      * rpc AssignRegion(.hbase.pb.AssignRegionRequest) returns (.hbase.pb.AssignRegionResponse);
      */
     public abstract void assignRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65839,9 +65839,9 @@ public final class MasterProtos {
      * rpc UnassignRegion(.hbase.pb.UnassignRegionRequest) returns (.hbase.pb.UnassignRegionResponse);
      */
     public abstract void unassignRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65855,9 +65855,9 @@ public final class MasterProtos {
      * rpc OfflineRegion(.hbase.pb.OfflineRegionRequest) returns (.hbase.pb.OfflineRegionResponse);
      */
     public abstract void offlineRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65867,9 +65867,9 @@ public final class MasterProtos {
      * rpc DeleteTable(.hbase.pb.DeleteTableRequest) returns (.hbase.pb.DeleteTableResponse);
      */
     public abstract void deleteTable(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65879,9 +65879,9 @@ public final class MasterProtos {
      * rpc truncateTable(.hbase.pb.TruncateTableRequest) returns (.hbase.pb.TruncateTableResponse);
      */
     public abstract void truncateTable(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65891,9 +65891,9 @@ public final class MasterProtos {
      * rpc EnableTable(.hbase.pb.EnableTableRequest) returns (.hbase.pb.EnableTableResponse);
      */
     public abstract void enableTable(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65903,9 +65903,9 @@ public final class MasterProtos {
      * rpc DisableTable(.hbase.pb.DisableTableRequest) returns (.hbase.pb.DisableTableResponse);
      */
     public abstract void disableTable(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65915,9 +65915,9 @@ public final class MasterProtos {
      * rpc ModifyTable(.hbase.pb.ModifyTableRequest) returns (.hbase.pb.ModifyTableResponse);
      */
     public abstract void modifyTable(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65927,9 +65927,9 @@ public final class MasterProtos {
      * rpc CreateTable(.hbase.pb.CreateTableRequest) returns (.hbase.pb.CreateTableResponse);
      */
     public abstract void createTable(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65939,9 +65939,9 @@ public final class MasterProtos {
      * rpc Shutdown(.hbase.pb.ShutdownRequest) returns (.hbase.pb.ShutdownResponse);
      */
     public abstract void shutdown(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65951,9 +65951,9 @@ public final class MasterProtos {
      * rpc StopMaster(.hbase.pb.StopMasterRequest) returns (.hbase.pb.StopMasterResponse);
      */
     public abstract void stopMaster(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65964,9 +65964,9 @@ public final class MasterProtos {
      * rpc IsMasterInMaintenanceMode(.hbase.pb.IsInMaintenanceModeRequest) returns (.hbase.pb.IsInMaintenanceModeResponse);
      */
     public abstract void isMasterInMaintenanceMode(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65979,9 +65979,9 @@ public final class MasterProtos {
      * rpc Balance(.hbase.pb.BalanceRequest) returns (.hbase.pb.BalanceResponse);
      */
     public abstract void balance(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -65993,9 +65993,9 @@ public final class MasterProtos {
      * rpc SetBalancerRunning(.hbase.pb.SetBalancerRunningRequest) returns (.hbase.pb.SetBalancerRunningResponse);
      */
     public abstract void setBalancerRunning(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66006,9 +66006,9 @@ public final class MasterProtos {
      * rpc IsBalancerEnabled(.hbase.pb.IsBalancerEnabledRequest) returns (.hbase.pb.IsBalancerEnabledResponse);
      */
     public abstract void isBalancerEnabled(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66020,9 +66020,9 @@ public final class MasterProtos {
      * rpc SetSplitOrMergeEnabled(.hbase.pb.SetSplitOrMergeEnabledRequest) returns (.hbase.pb.SetSplitOrMergeEnabledResponse);
      */
     public abstract void setSplitOrMergeEnabled(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66033,9 +66033,9 @@ public final class MasterProtos {
      * rpc IsSplitOrMergeEnabled(.hbase.pb.IsSplitOrMergeEnabledRequest) returns (.hbase.pb.IsSplitOrMergeEnabledResponse);
      */
     public abstract void isSplitOrMergeEnabled(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66046,9 +66046,9 @@ public final class MasterProtos {
      * rpc Normalize(.hbase.pb.NormalizeRequest) returns (.hbase.pb.NormalizeResponse);
      */
     public abstract void normalize(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66059,9 +66059,9 @@ public final class MasterProtos {
      * rpc SetNormalizerRunning(.hbase.pb.SetNormalizerRunningRequest) returns (.hbase.pb.SetNormalizerRunningResponse);
      */
     public abstract void setNormalizerRunning(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66072,9 +66072,9 @@ public final class MasterProtos {
      * rpc IsNormalizerEnabled(.hbase.pb.IsNormalizerEnabledRequest) returns (.hbase.pb.IsNormalizerEnabledResponse);
      */
     public abstract void isNormalizerEnabled(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66084,9 +66084,9 @@ public final class MasterProtos {
      * rpc RunCatalogScan(.hbase.pb.RunCatalogScanRequest) returns (.hbase.pb.RunCatalogScanResponse);
      */
     public abstract void runCatalogScan(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66097,9 +66097,9 @@ public final class MasterProtos {
      * rpc EnableCatalogJanitor(.hbase.pb.EnableCatalogJanitorRequest) returns (.hbase.pb.EnableCatalogJanitorResponse);
      */
     public abstract void enableCatalogJanitor(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66110,9 +66110,9 @@ public final class MasterProtos {
      * rpc IsCatalogJanitorEnabled(.hbase.pb.IsCatalogJanitorEnabledRequest) returns (.hbase.pb.IsCatalogJanitorEnabledResponse);
      */
     public abstract void isCatalogJanitorEnabled(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66123,9 +66123,9 @@ public final class MasterProtos {
      * rpc ExecMasterService(.hbase.pb.CoprocessorServiceRequest) returns (.hbase.pb.CoprocessorServiceResponse);
      */
     public abstract void execMasterService(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66136,9 +66136,9 @@ public final class MasterProtos {
      * rpc Snapshot(.hbase.pb.SnapshotRequest) returns (.hbase.pb.SnapshotResponse);
      */
     public abstract void snapshot(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66150,9 +66150,9 @@ public final class MasterProtos {
      * rpc GetCompletedSnapshots(.hbase.pb.GetCompletedSnapshotsRequest) returns (.hbase.pb.GetCompletedSnapshotsResponse);
      */
     public abstract void getCompletedSnapshots(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66163,9 +66163,9 @@ public final class MasterProtos {
      * rpc DeleteSnapshot(.hbase.pb.DeleteSnapshotRequest) returns (.hbase.pb.DeleteSnapshotResponse);
      */
     public abstract void deleteSnapshot(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66176,9 +66176,9 @@ public final class MasterProtos {
      * rpc IsSnapshotDone(.hbase.pb.IsSnapshotDoneRequest) returns (.hbase.pb.IsSnapshotDoneResponse);
      */
     public abstract void isSnapshotDone(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66189,9 +66189,9 @@ public final class MasterProtos {
      * rpc RestoreSnapshot(.hbase.pb.RestoreSnapshotRequest) returns (.hbase.pb.RestoreSnapshotResponse);
      */
     public abstract void restoreSnapshot(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66202,9 +66202,9 @@ public final class MasterProtos {
      * rpc ExecProcedure(.hbase.pb.ExecProcedureRequest) returns (.hbase.pb.ExecProcedureResponse);
      */
     public abstract void execProcedure(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66215,9 +66215,9 @@ public final class MasterProtos {
      * rpc ExecProcedureWithRet(.hbase.pb.ExecProcedureRequest) returns (.hbase.pb.ExecProcedureResponse);
      */
     public abstract void execProcedureWithRet(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66228,9 +66228,9 @@ public final class MasterProtos {
      * rpc IsProcedureDone(.hbase.pb.IsProcedureDoneRequest) returns (.hbase.pb.IsProcedureDoneResponse);
      */
     public abstract void isProcedureDone(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66240,9 +66240,9 @@ public final class MasterProtos {
      * rpc ModifyNamespace(.hbase.pb.ModifyNamespaceRequest) returns (.hbase.pb.ModifyNamespaceResponse);
      */
     public abstract void modifyNamespace(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66252,9 +66252,9 @@ public final class MasterProtos {
      * rpc CreateNamespace(.hbase.pb.CreateNamespaceRequest) returns (.hbase.pb.CreateNamespaceResponse);
      */
     public abstract void createNamespace(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66264,9 +66264,9 @@ public final class MasterProtos {
      * rpc DeleteNamespace(.hbase.pb.DeleteNamespaceRequest) returns (.hbase.pb.DeleteNamespaceResponse);
      */
     public abstract void deleteNamespace(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66276,9 +66276,9 @@ public final class MasterProtos {
      * rpc GetNamespaceDescriptor(.hbase.pb.GetNamespaceDescriptorRequest) returns (.hbase.pb.GetNamespaceDescriptorResponse);
      */
     public abstract void getNamespaceDescriptor(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66288,9 +66288,9 @@ public final class MasterProtos {
      * rpc ListNamespaceDescriptors(.hbase.pb.ListNamespaceDescriptorsRequest) returns (.hbase.pb.ListNamespaceDescriptorsResponse);
      */
     public abstract void listNamespaceDescriptors(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66300,9 +66300,9 @@ public final class MasterProtos {
      * rpc ListTableDescriptorsByNamespace(.hbase.pb.ListTableDescriptorsByNamespaceRequest) returns (.hbase.pb.ListTableDescriptorsByNamespaceResponse);
      */
     public abstract void listTableDescriptorsByNamespace(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66312,9 +66312,9 @@ public final class MasterProtos {
      * rpc ListTableNamesByNamespace(.hbase.pb.ListTableNamesByNamespaceRequest) returns (.hbase.pb.ListTableNamesByNamespaceResponse);
      */
     public abstract void listTableNamesByNamespace(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66324,9 +66324,9 @@ public final class MasterProtos {
      * rpc GetTableState(.hbase.pb.GetTableStateRequest) returns (.hbase.pb.GetTableStateResponse);
      */
     public abstract void getTableState(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66336,9 +66336,9 @@ public final class MasterProtos {
      * rpc SetQuota(.hbase.pb.SetQuotaRequest) returns (.hbase.pb.SetQuotaResponse);
      */
     public abstract void setQuota(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66348,9 +66348,9 @@ public final class MasterProtos {
      * rpc getLastMajorCompactionTimestamp(.hbase.pb.MajorCompactionTimestampRequest) returns (.hbase.pb.MajorCompactionTimestampResponse);
      */
     public abstract void getLastMajorCompactionTimestamp(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66360,17 +66360,17 @@ public final class MasterProtos {
      * rpc getLastMajorCompactionTimestampForRegion(.hbase.pb.MajorCompactionTimestampForRegionRequest) returns (.hbase.pb.MajorCompactionTimestampResponse);
      */
     public abstract void getLastMajorCompactionTimestampForRegion(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * rpc getProcedureResult(.hbase.pb.GetProcedureResultRequest) returns (.hbase.pb.GetProcedureResultResponse);
      */
     public abstract void getProcedureResult(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66380,9 +66380,9 @@ public final class MasterProtos {
      * rpc getSecurityCapabilities(.hbase.pb.SecurityCapabilitiesRequest) returns (.hbase.pb.SecurityCapabilitiesResponse);
      */
     public abstract void getSecurityCapabilities(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66392,9 +66392,9 @@ public final class MasterProtos {
      * rpc AbortProcedure(.hbase.pb.AbortProcedureRequest) returns (.hbase.pb.AbortProcedureResponse);
      */
     public abstract void abortProcedure(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -66404,26 +66404,26 @@ public final class MasterProtos {
      * rpc ListProcedures(.hbase.pb.ListProceduresRequest) returns (.hbase.pb.ListProceduresResponse);
      */
     public abstract void listProcedures(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.getDescriptor().getServices().get(0);
     }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
 
     public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Message> done) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.callMethod() given method descriptor for wrong " +
@@ -66432,287 +66432,287 @@ public final class MasterProtos {
       switch(method.getIndex()) {
         case 0:
           this.getSchemaAlterStatus(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 1:
           this.getTableDescriptors(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 2:
           this.getTableNames(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 3:
           this.getClusterStatus(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 4:
           this.isMasterRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 5:
           this.addColumn(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 6:
           this.deleteColumn(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 7:
           this.modifyColumn(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 8:
           this.moveRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 9:
           this.dispatchMergingRegions(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 10:
           this.assignRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 11:
           this.unassignRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 12:
           this.offlineRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 13:
           this.deleteTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 14:
           this.truncateTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 15:
           this.enableTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 16:
           this.disableTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 17:
           this.modifyTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 18:
           this.createTable(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 19:
           this.shutdown(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 20:
           this.stopMaster(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 21:
           this.isMasterInMaintenanceMode(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 22:
           this.balance(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 23:
           this.setBalancerRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 24:
           this.isBalancerEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 25:
           this.setSplitOrMergeEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 26:
           this.isSplitOrMergeEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 27:
           this.normalize(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 28:
           this.setNormalizerRunning(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 29:
           this.isNormalizerEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 30:
           this.runCatalogScan(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 31:
           this.enableCatalogJanitor(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 32:
           this.isCatalogJanitorEnabled(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 33:
           this.execMasterService(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 34:
           this.snapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 35:
           this.getCompletedSnapshots(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 36:
           this.deleteSnapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 37:
           this.isSnapshotDone(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 38:
           this.restoreSnapshot(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 39:
           this.execProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 40:
           this.execProcedureWithRet(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 41:
           this.isProcedureDone(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 42:
           this.modifyNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 43:
           this.createNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 44:
           this.deleteNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 45:
           this.getNamespaceDescriptor(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 46:
           this.listNamespaceDescriptors(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 47:
           this.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 48:
           this.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 49:
           this.getTableState(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 50:
           this.setQuota(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 51:
           this.getLastMajorCompactionTimestamp(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 52:
           this.getLastMajorCompactionTimestampForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 53:
           this.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 54:
           this.getSecurityCapabilities(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 55:
           this.abortProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 56:
           this.listProcedures(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         default:
@@ -66720,9 +66720,9 @@ public final class MasterProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getRequestPrototype() given method " +
@@ -66848,9 +66848,9 @@ public final class MasterProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getResponsePrototype() given method " +
@@ -66977,871 +66977,871 @@ public final class MasterProtos {
     }
 
     public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
       return new Stub(channel);
     }
 
     public static final class Stub extends org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
+      private Stub(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.RpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel;
 
-      public com.google.protobuf.RpcChannel getChannel() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel getChannel() {
         return channel;
       }
 
       public  void getSchemaAlterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(0),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse.getDefaultInstance()));
       }
 
       public  void getTableDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse.getDefaultInstance()));
       }
 
       public  void getTableNames(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(2),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse.getDefaultInstance()));
       }
 
       public  void getClusterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(3),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse.getDefaultInstance()));
       }
 
       public  void isMasterRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(4),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse.getDefaultInstance()));
       }
 
       public  void addColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(5),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse.getDefaultInstance()));
       }
 
       public  void deleteColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(6),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse.getDefaultInstance()));
       }
 
       public  void modifyColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(7),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse.getDefaultInstance()));
       }
 
       public  void moveRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(8),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse.getDefaultInstance()));
       }
 
       public  void dispatchMergingRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(9),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.getDefaultInstance()));
       }
 
       public  void assignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(10),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse.getDefaultInstance()));
       }
 
       public  void unassignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(11),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse.getDefaultInstance()));
       }
 
       public  void offlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(12),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse.getDefaultInstance()));
       }
 
       public  void deleteTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(13),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse.getDefaultInstance()));
       }
 
       public  void truncateTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(14),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance()));
       }
 
       public  void enableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(15),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse.getDefaultInstance()));
       }
 
       public  void disableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(16),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse.getDefaultInstance()));
       }
 
       public  void modifyTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(17),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse.getDefaultInstance()));
       }
 
       public  void createTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(18),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse.getDefaultInstance()));
       }
 
       public  void shutdown(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(19),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse.getDefaultInstance()));
       }
 
       public  void stopMaster(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(20),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse.getDefaultInstance()));
       }
 
       public  void isMasterInMaintenanceMode(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(21),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse.getDefaultInstance()));
       }
 
       public  void balance(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(22),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse.getDefaultInstance()));
       }
 
       public  void setBalancerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(23),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse.getDefaultInstance()));
       }
 
       public  void isBalancerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(24),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse.getDefaultInstance()));
       }
 
       public  void setSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(25),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.getDefaultInstance()));
       }
 
       public  void isSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(26),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse.getDefaultInstance()));
       }
 
       public  void normalize(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(27),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse.getDefaultInstance()));
       }
 
       public  void setNormalizerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(28),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse.getDefaultInstance()));
       }
 
       public  void isNormalizerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(29),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse.getDefaultInstance()));
       }
 
       public  void runCatalogScan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(30),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse.getDefaultInstance()));
       }
 
       public  void enableCatalogJanitor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(31),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse.getDefaultInstance()));
       }
 
       public  void isCatalogJanitorEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(32),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance()));
       }
 
       public  void execMasterService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(33),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()));
       }
 
       public  void snapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(34),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse.getDefaultInstance()));
       }
 
       public  void getCompletedSnapshots(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(35),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse.getDefaultInstance()));
       }
 
       public  void deleteSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(36),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse.getDefaultInstance()));
       }
 
       public  void isSnapshotDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(37),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse.getDefaultInstance()));
       }
 
       public  void restoreSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(38),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse.getDefaultInstance()));
       }
 
       public  void execProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(39),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse.getDefaultInstance()));
       }
 
       public  void execProcedureWithRet(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(40),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse.getDefaultInstance()));
       }
 
       public  void isProcedureDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(41),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse.getDefaultInstance()));
       }
 
       public  void modifyNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(42),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse.getDefaultInstance()));
       }
 
       public  void createNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(43),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse.getDefaultInstance()));
       }
 
       public  void deleteNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(44),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse.getDefaultInstance()));
       }
 
       public  void getNamespaceDescriptor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(45),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse.getDefaultInstance()));
       }
 
       public  void listNamespaceDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(46),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse.getDefaultInstance()));
       }
 
       public  void listTableDescriptorsByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(47),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance()));
       }
 
       public  void listTableNamesByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(48),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance()));
       }
 
       public  void getTableState(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(49),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance()));
       }
 
       public  void setQuota(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(50),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance()));
       }
 
       public  void getLastMajorCompactionTimestamp(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(51),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance()));
       }
 
       public  void getLastMajorCompactionTimestampForRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(52),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance()));
       }
 
       public  void getProcedureResult(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(53),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance()));
       }
 
       public  void getSecurityCapabilities(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(54),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance()));
       }
 
       public  void abortProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(55),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance()));
       }
 
       public  void listProcedures(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(56),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance()));
@@ -67849,308 +67849,308 @@ public final class MasterProtos {
     }
 
     public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
       return new BlockingStub(channel);
     }
 
     public interface BlockingInterface {
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse getSchemaAlterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse getTableDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse getTableNames(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse getClusterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse isMasterRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse addColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse deleteColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse modifyColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse moveRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse dispatchMergingRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse assignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse unassignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse offlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse deleteTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse truncateTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse enableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse disableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse modifyTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse createTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse shutdown(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse stopMaster(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse isMasterInMaintenanceMode(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse balance(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse setBalancerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse isBalancerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse normalize(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse setNormalizerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse isNormalizerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse runCatalogScan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse enableCatalogJanitor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execMasterService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse snapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse getCompletedSnapshots(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse deleteSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse isSnapshotDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse restoreSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse execProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse execProcedureWithRet(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse isProcedureDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse modifyNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse createNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse deleteNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse listNamespaceDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse listTableNamesByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse getTableState(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse setQuota(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse getProcedureResult(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse getSecurityCapabilities(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse abortProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse listProcedures(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+      private BlockingStub(org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.BlockingRpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse getSchemaAlterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(0),
           controller,
@@ -68160,9 +68160,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse getTableDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(1),
           controller,
@@ -68172,9 +68172,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse getTableNames(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(2),
           controller,
@@ -68184,9 +68184,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse getClusterStatus(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(3),
           controller,
@@ -68196,9 +68196,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse isMasterRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(4),
           controller,
@@ -68208,9 +68208,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse addColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(5),
           controller,
@@ -68220,9 +68220,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse deleteColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(6),
           controller,
@@ -68232,9 +68232,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse modifyColumn(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(7),
           controller,
@@ -68244,9 +68244,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse moveRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(8),
           controller,
@@ -68256,9 +68256,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse dispatchMergingRegions(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(9),
           controller,
@@ -68268,9 +68268,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse assignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(10),
           controller,
@@ -68280,9 +68280,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse unassignRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(11),
           controller,
@@ -68292,9 +68292,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse offlineRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(12),
           controller,
@@ -68304,9 +68304,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse deleteTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(13),
           controller,
@@ -68316,9 +68316,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse truncateTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(14),
           controller,
@@ -68328,9 +68328,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse enableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(15),
           controller,
@@ -68340,9 +68340,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse disableTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(16),
           controller,
@@ -68352,9 +68352,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse modifyTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(17),
           controller,
@@ -68364,9 +68364,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse createTable(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(18),
           controller,
@@ -68376,9 +68376,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse shutdown(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(19),
           controller,
@@ -68388,9 +68388,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse stopMaster(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(20),
           controller,
@@ -68400,9 +68400,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse isMasterInMaintenanceMode(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(21),
           controller,
@@ -68412,9 +68412,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse balance(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(22),
           controller,
@@ -68424,9 +68424,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse setBalancerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(23),
           controller,
@@ -68436,9 +68436,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse isBalancerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(24),
           controller,
@@ -68448,9 +68448,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(25),
           controller,
@@ -68460,9 +68460,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(26),
           controller,
@@ -68472,9 +68472,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse normalize(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(27),
           controller,
@@ -68484,9 +68484,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse setNormalizerRunning(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(28),
           controller,
@@ -68496,9 +68496,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse isNormalizerEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(29),
           controller,
@@ -68508,9 +68508,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse runCatalogScan(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(30),
           controller,
@@ -68520,9 +68520,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse enableCatalogJanitor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(31),
           controller,
@@ -68532,9 +68532,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(32),
           controller,
@@ -68544,9 +68544,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse execMasterService(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(33),
           controller,
@@ -68556,9 +68556,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse snapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(34),
           controller,
@@ -68568,9 +68568,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse getCompletedSnapshots(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(35),
           controller,
@@ -68580,9 +68580,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse deleteSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(36),
           controller,
@@ -68592,9 +68592,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse isSnapshotDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(37),
           controller,
@@ -68604,9 +68604,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse restoreSnapshot(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(38),
           controller,
@@ -68616,9 +68616,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse execProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(39),
           controller,
@@ -68628,9 +68628,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse execProcedureWithRet(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(40),
           controller,
@@ -68640,9 +68640,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse isProcedureDone(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(41),
           controller,
@@ -68652,9 +68652,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse modifyNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(42),
           controller,
@@ -68664,9 +68664,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse createNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(43),
           controller,
@@ -68676,9 +68676,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse deleteNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(44),
           controller,
@@ -68688,9 +68688,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(45),
           controller,
@@ -68700,9 +68700,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse listNamespaceDescriptors(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(46),
           controller,
@@ -68712,9 +68712,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(47),
           controller,
@@ -68724,9 +68724,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse listTableNamesByNamespace(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(48),
           controller,
@@ -68736,9 +68736,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse getTableState(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(49),
           controller,
@@ -68748,9 +68748,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse setQuota(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(50),
           controller,
@@ -68760,9 +68760,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(51),
           controller,
@@ -68772,9 +68772,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(52),
           controller,
@@ -68784,9 +68784,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse getProcedureResult(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(53),
           controller,
@@ -68796,9 +68796,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse getSecurityCapabilities(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(54),
           controller,
@@ -68808,9 +68808,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse abortProcedure(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(55),
           controller,
@@ -68820,9 +68820,9 @@ public final class MasterProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse listProcedures(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(56),
           controller,
@@ -68835,567 +68835,567 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.MasterService)
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AddColumnRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AddColumnRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AddColumnResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AddColumnResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteColumnRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteColumnRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteColumnResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteColumnResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyColumnRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyColumnRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyColumnResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyColumnResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MoveRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MoveRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MoveRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MoveRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AssignRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AssignRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AssignRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AssignRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UnassignRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UnassignRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UnassignRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UnassignRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_OfflineRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_OfflineRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_OfflineRegionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_OfflineRegionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CreateTableRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CreateTableRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CreateTableResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CreateTableResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteTableRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteTableRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteTableResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteTableResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TruncateTableRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TruncateTableRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TruncateTableResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TruncateTableResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_EnableTableRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_EnableTableRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_EnableTableResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_EnableTableResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DisableTableRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DisableTableRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DisableTableResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DisableTableResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyTableRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyTableRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyTableResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyTableResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CreateNamespaceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CreateNamespaceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CreateNamespaceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteNamespaceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteNamespaceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyNamespaceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ModifyNamespaceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetNamespaceDescriptorRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetNamespaceDescriptorResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetNamespaceDescriptorResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListNamespaceDescriptorsRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListNamespaceDescriptorsRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListNamespaceDescriptorsResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListNamespaceDescriptorsResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListTableNamesByNamespaceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListTableNamesByNamespaceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListTableNamesByNamespaceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListTableNamesByNamespaceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ShutdownRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ShutdownRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ShutdownResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ShutdownResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StopMasterRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StopMasterRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StopMasterResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StopMasterResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsInMaintenanceModeRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsInMaintenanceModeRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsInMaintenanceModeResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsInMaintenanceModeResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BalanceRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BalanceRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BalanceResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BalanceResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetBalancerRunningRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetBalancerRunningRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetBalancerRunningResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetBalancerRunningResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsBalancerEnabledRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsBalancerEnabledRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsBalancerEnabledResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsBalancerEnabledResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NormalizeRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NormalizeResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetNormalizerRunningResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetNormalizerRunningResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsNormalizerEnabledRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsNormalizerEnabledRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsNormalizerEnabledResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsNormalizerEnabledResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RunCatalogScanRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RunCatalogScanRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RunCatalogScanResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RunCatalogScanResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_EnableCatalogJanitorRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_EnableCatalogJanitorRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_EnableCatalogJanitorResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_EnableCatalogJanitorResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetCompletedSnapshotsRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetCompletedSnapshotsRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetCompletedSnapshotsResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetCompletedSnapshotsResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteSnapshotRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteSnapshotRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeleteSnapshotResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeleteSnapshotResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RestoreSnapshotRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RestoreSnapshotRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RestoreSnapshotResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RestoreSnapshotResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsSnapshotDoneRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsSnapshotDoneRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsSnapshotDoneResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsSnapshotDoneResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSchemaAlterStatusRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSchemaAlterStatusRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSchemaAlterStatusResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSchemaAlterStatusResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetTableDescriptorsRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetTableDescriptorsRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetTableDescriptorsResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetTableDescriptorsResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetTableNamesRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetTableNamesRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetTableNamesResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetTableNamesResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetTableStateRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetTableStateRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetTableStateResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetTableStateResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetClusterStatusRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetClusterStatusRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetClusterStatusResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetClusterStatusResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsMasterRunningRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsMasterRunningRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsMasterRunningResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsMasterRunningResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ExecProcedureRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ExecProcedureRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ExecProcedureResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ExecProcedureResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsProcedureDoneRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsProcedureDoneRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsProcedureDoneResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsProcedureDoneResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetProcedureResultRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetProcedureResultRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetProcedureResultResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetProcedureResultResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AbortProcedureRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AbortProcedureRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_AbortProcedureResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_AbortProcedureResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListProceduresRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListProceduresRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ListProceduresResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ListProceduresResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetQuotaRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetQuotaRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetQuotaResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MajorCompactionTimestampRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MajorCompactionTimestampRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MajorCompactionTimestampResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MajorCompactionTimestampResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SecurityCapabilitiesRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SecurityCapabilitiesRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -69730,17 +69730,17 @@ public final class MasterProtos {
       "e.hadoop.hbase.shaded.protobuf.generated" +
       "B\014MasterProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(),
@@ -69751,667 +69751,667 @@ public final class MasterProtos {
     internal_static_hbase_pb_AddColumnRequest_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_AddColumnRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AddColumnRequest_descriptor,
         new java.lang.String[] { "TableName", "ColumnFamilies", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_AddColumnResponse_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_AddColumnResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AddColumnResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_DeleteColumnRequest_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_DeleteColumnRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteColumnRequest_descriptor,
         new java.lang.String[] { "TableName", "ColumnName", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_DeleteColumnResponse_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_DeleteColumnResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteColumnResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_ModifyColumnRequest_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_ModifyColumnRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyColumnRequest_descriptor,
         new java.lang.String[] { "TableName", "ColumnFamilies", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_ModifyColumnResponse_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_ModifyColumnResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyColumnResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_MoveRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_MoveRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MoveRegionRequest_descriptor,
         new java.lang.String[] { "Region", "DestServerName", });
     internal_static_hbase_pb_MoveRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_MoveRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MoveRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor,
         new java.lang.String[] { "RegionA", "RegionB", "Forcible", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_AssignRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_AssignRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AssignRegionRequest_descriptor,
         new java.lang.String[] { "Region", });
     internal_static_hbase_pb_AssignRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(11);
     internal_static_hbase_pb_AssignRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AssignRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_UnassignRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_UnassignRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UnassignRegionRequest_descriptor,
         new java.lang.String[] { "Region", "Force", });
     internal_static_hbase_pb_UnassignRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_hbase_pb_UnassignRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UnassignRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_OfflineRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(14);
     internal_static_hbase_pb_OfflineRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_OfflineRegionRequest_descriptor,
         new java.lang.String[] { "Region", });
     internal_static_hbase_pb_OfflineRegionResponse_descriptor =
       getDescriptor().getMessageTypes().get(15);
     internal_static_hbase_pb_OfflineRegionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_OfflineRegionResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_CreateTableRequest_descriptor =
       getDescriptor().getMessageTypes().get(16);
     internal_static_hbase_pb_CreateTableRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CreateTableRequest_descriptor,
         new java.lang.String[] { "TableSchema", "SplitKeys", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_CreateTableResponse_descriptor =
       getDescriptor().getMessageTypes().get(17);
     internal_static_hbase_pb_CreateTableResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CreateTableResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_DeleteTableRequest_descriptor =
       getDescriptor().getMessageTypes().get(18);
     internal_static_hbase_pb_DeleteTableRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteTableRequest_descriptor,
         new java.lang.String[] { "TableName", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_DeleteTableResponse_descriptor =
       getDescriptor().getMessageTypes().get(19);
     internal_static_hbase_pb_DeleteTableResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteTableResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_TruncateTableRequest_descriptor =
       getDescriptor().getMessageTypes().get(20);
     internal_static_hbase_pb_TruncateTableRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TruncateTableRequest_descriptor,
         new java.lang.String[] { "TableName", "PreserveSplits", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_TruncateTableResponse_descriptor =
       getDescriptor().getMessageTypes().get(21);
     internal_static_hbase_pb_TruncateTableResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TruncateTableResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_EnableTableRequest_descriptor =
       getDescriptor().getMessageTypes().get(22);
     internal_static_hbase_pb_EnableTableRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_EnableTableRequest_descriptor,
         new java.lang.String[] { "TableName", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_EnableTableResponse_descriptor =
       getDescriptor().getMessageTypes().get(23);
     internal_static_hbase_pb_EnableTableResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_EnableTableResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_DisableTableRequest_descriptor =
       getDescriptor().getMessageTypes().get(24);
     internal_static_hbase_pb_DisableTableRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DisableTableRequest_descriptor,
         new java.lang.String[] { "TableName", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_DisableTableResponse_descriptor =
       getDescriptor().getMessageTypes().get(25);
     internal_static_hbase_pb_DisableTableResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DisableTableResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_ModifyTableRequest_descriptor =
       getDescriptor().getMessageTypes().get(26);
     internal_static_hbase_pb_ModifyTableRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyTableRequest_descriptor,
         new java.lang.String[] { "TableName", "TableSchema", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_ModifyTableResponse_descriptor =
       getDescriptor().getMessageTypes().get(27);
     internal_static_hbase_pb_ModifyTableResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyTableResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_CreateNamespaceRequest_descriptor =
       getDescriptor().getMessageTypes().get(28);
     internal_static_hbase_pb_CreateNamespaceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CreateNamespaceRequest_descriptor,
         new java.lang.String[] { "NamespaceDescriptor", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_CreateNamespaceResponse_descriptor =
       getDescriptor().getMessageTypes().get(29);
     internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CreateNamespaceResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_DeleteNamespaceRequest_descriptor =
       getDescriptor().getMessageTypes().get(30);
     internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteNamespaceRequest_descriptor,
         new java.lang.String[] { "NamespaceName", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_DeleteNamespaceResponse_descriptor =
       getDescriptor().getMessageTypes().get(31);
     internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteNamespaceResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_ModifyNamespaceRequest_descriptor =
       getDescriptor().getMessageTypes().get(32);
     internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyNamespaceRequest_descriptor,
         new java.lang.String[] { "NamespaceDescriptor", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_ModifyNamespaceResponse_descriptor =
       getDescriptor().getMessageTypes().get(33);
     internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ModifyNamespaceResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor =
       getDescriptor().getMessageTypes().get(34);
     internal_static_hbase_pb_GetNamespaceDescriptorRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor,
         new java.lang.String[] { "NamespaceName", });
     internal_static_hbase_pb_GetNamespaceDescriptorResponse_descriptor =
       getDescriptor().getMessageTypes().get(35);
     internal_static_hbase_pb_GetNamespaceDescriptorResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetNamespaceDescriptorResponse_descriptor,
         new java.lang.String[] { "NamespaceDescriptor", });
     internal_static_hbase_pb_ListNamespaceDescriptorsRequest_descriptor =
       getDescriptor().getMessageTypes().get(36);
     internal_static_hbase_pb_ListNamespaceDescriptorsRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListNamespaceDescriptorsRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_ListNamespaceDescriptorsResponse_descriptor =
       getDescriptor().getMessageTypes().get(37);
     internal_static_hbase_pb_ListNamespaceDescriptorsResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListNamespaceDescriptorsResponse_descriptor,
         new java.lang.String[] { "NamespaceDescriptor", });
     internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_descriptor =
       getDescriptor().getMessageTypes().get(38);
     internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListTableDescriptorsByNamespaceRequest_descriptor,
         new java.lang.String[] { "NamespaceName", });
     internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_descriptor =
       getDescriptor().getMessageTypes().get(39);
     internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListTableDescriptorsByNamespaceResponse_descriptor,
         new java.lang.String[] { "TableSchema", });
     internal_static_hbase_pb_ListTableNamesByNamespaceRequest_descriptor =
       getDescriptor().getMessageTypes().get(40);
     internal_static_hbase_pb_ListTableNamesByNamespaceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListTableNamesByNamespaceRequest_descriptor,
         new java.lang.String[] { "NamespaceName", });
     internal_static_hbase_pb_ListTableNamesByNamespaceResponse_descriptor =
       getDescriptor().getMessageTypes().get(41);
     internal_static_hbase_pb_ListTableNamesByNamespaceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListTableNamesByNamespaceResponse_descriptor,
         new java.lang.String[] { "TableName", });
     internal_static_hbase_pb_ShutdownRequest_descriptor =
       getDescriptor().getMessageTypes().get(42);
     internal_static_hbase_pb_ShutdownRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ShutdownRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_ShutdownResponse_descriptor =
       getDescriptor().getMessageTypes().get(43);
     internal_static_hbase_pb_ShutdownResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ShutdownResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_StopMasterRequest_descriptor =
       getDescriptor().getMessageTypes().get(44);
     internal_static_hbase_pb_StopMasterRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StopMasterRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_StopMasterResponse_descriptor =
       getDescriptor().getMessageTypes().get(45);
     internal_static_hbase_pb_StopMasterResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StopMasterResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_IsInMaintenanceModeRequest_descriptor =
       getDescriptor().getMessageTypes().get(46);
     internal_static_hbase_pb_IsInMaintenanceModeRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsInMaintenanceModeRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_IsInMaintenanceModeResponse_descriptor =
       getDescriptor().getMessageTypes().get(47);
     internal_static_hbase_pb_IsInMaintenanceModeResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsInMaintenanceModeResponse_descriptor,
         new java.lang.String[] { "InMaintenanceMode", });
     internal_static_hbase_pb_BalanceRequest_descriptor =
       getDescriptor().getMessageTypes().get(48);
     internal_static_hbase_pb_BalanceRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BalanceRequest_descriptor,
         new java.lang.String[] { "Force", });
     internal_static_hbase_pb_BalanceResponse_descriptor =
       getDescriptor().getMessageTypes().get(49);
     internal_static_hbase_pb_BalanceResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BalanceResponse_descriptor,
         new java.lang.String[] { "BalancerRan", });
     internal_static_hbase_pb_SetBalancerRunningRequest_descriptor =
       getDescriptor().getMessageTypes().get(50);
     internal_static_hbase_pb_SetBalancerRunningRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetBalancerRunningRequest_descriptor,
         new java.lang.String[] { "On", "Synchronous", });
     internal_static_hbase_pb_SetBalancerRunningResponse_descriptor =
       getDescriptor().getMessageTypes().get(51);
     internal_static_hbase_pb_SetBalancerRunningResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetBalancerRunningResponse_descriptor,
         new java.lang.String[] { "PrevBalanceValue", });
     internal_static_hbase_pb_IsBalancerEnabledRequest_descriptor =
       getDescriptor().getMessageTypes().get(52);
     internal_static_hbase_pb_IsBalancerEnabledRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsBalancerEnabledRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_IsBalancerEnabledResponse_descriptor =
       getDescriptor().getMessageTypes().get(53);
     internal_static_hbase_pb_IsBalancerEnabledResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsBalancerEnabledResponse_descriptor,
         new java.lang.String[] { "Enabled", });
     internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor =
       getDescriptor().getMessageTypes().get(54);
     internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor,
         new java.lang.String[] { "Enabled", "Synchronous", "SwitchTypes", });
     internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor =
       getDescriptor().getMessageTypes().get(55);
     internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor,
         new java.lang.String[] { "PrevValue", });
     internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor =
       getDescriptor().getMessageTypes().get(56);
     internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor,
         new java.lang.String[] { "SwitchType", });
     internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_descriptor =
       getDescriptor().getMessageTypes().get(57);
     internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsSplitOrMergeEnabledResponse_descriptor,
         new java.lang.String[] { "Enabled", });
     internal_static_hbase_pb_NormalizeRequest_descriptor =
       getDescriptor().getMessageTypes().get(58);
     internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NormalizeRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_NormalizeResponse_descriptor =
       getDescriptor().getMessageTypes().get(59);
     internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_NormalizeResponse_descriptor,
         new java.lang.String[] { "NormalizerRan", });
     internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor =
       getDescriptor().getMessageTypes().get(60);
     internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor,
         new java.lang.String[] { "On", });
     internal_static_hbase_pb_SetNormalizerRunningResponse_descriptor =
       getDescriptor().getMessageTypes().get(61);
     internal_static_hbase_pb_SetNormalizerRunningResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetNormalizerRunningResponse_descriptor,
         new java.lang.String[] { "PrevNormalizerValue", });
     internal_static_hbase_pb_IsNormalizerEnabledRequest_descriptor =
       getDescriptor().getMessageTypes().get(62);
     internal_static_hbase_pb_IsNormalizerEnabledRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsNormalizerEnabledRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_IsNormalizerEnabledResponse_descriptor =
       getDescriptor().getMessageTypes().get(63);
     internal_static_hbase_pb_IsNormalizerEnabledResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsNormalizerEnabledResponse_descriptor,
         new java.lang.String[] { "Enabled", });
     internal_static_hbase_pb_RunCatalogScanRequest_descriptor =
       getDescriptor().getMessageTypes().get(64);
     internal_static_hbase_pb_RunCatalogScanRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RunCatalogScanRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_RunCatalogScanResponse_descriptor =
       getDescriptor().getMessageTypes().get(65);
     internal_static_hbase_pb_RunCatalogScanResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RunCatalogScanResponse_descriptor,
         new java.lang.String[] { "ScanResult", });
     internal_static_hbase_pb_EnableCatalogJanitorRequest_descriptor =
       getDescriptor().getMessageTypes().get(66);
     internal_static_hbase_pb_EnableCatalogJanitorRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_EnableCatalogJanitorRequest_descriptor,
         new java.lang.String[] { "Enable", });
     internal_static_hbase_pb_EnableCatalogJanitorResponse_descriptor =
       getDescriptor().getMessageTypes().get(67);
     internal_static_hbase_pb_EnableCatalogJanitorResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_EnableCatalogJanitorResponse_descriptor,
         new java.lang.String[] { "PrevValue", });
     internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_descriptor =
       getDescriptor().getMessageTypes().get(68);
     internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsCatalogJanitorEnabledRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_descriptor =
       getDescriptor().getMessageTypes().get(69);
     internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_descriptor,
         new java.lang.String[] { "Value", });
     internal_static_hbase_pb_SnapshotRequest_descriptor =
       getDescriptor().getMessageTypes().get(70);
     internal_static_hbase_pb_SnapshotRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotRequest_descriptor,
         new java.lang.String[] { "Snapshot", });
     internal_static_hbase_pb_SnapshotResponse_descriptor =
       getDescriptor().getMessageTypes().get(71);
     internal_static_hbase_pb_SnapshotResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotResponse_descriptor,
         new java.lang.String[] { "ExpectedTimeout", });
     internal_static_hbase_pb_GetCompletedSnapshotsRequest_descriptor =
       getDescriptor().getMessageTypes().get(72);
     internal_static_hbase_pb_GetCompletedSnapshotsRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetCompletedSnapshotsRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_GetCompletedSnapshotsResponse_descriptor =
       getDescriptor().getMessageTypes().get(73);
     internal_static_hbase_pb_GetCompletedSnapshotsResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetCompletedSnapshotsResponse_descriptor,
         new java.lang.String[] { "Snapshots", });
     internal_static_hbase_pb_DeleteSnapshotRequest_descriptor =
       getDescriptor().getMessageTypes().get(74);
     internal_static_hbase_pb_DeleteSnapshotRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteSnapshotRequest_descriptor,
         new java.lang.String[] { "Snapshot", });
     internal_static_hbase_pb_DeleteSnapshotResponse_descriptor =
       getDescriptor().getMessageTypes().get(75);
     internal_static_hbase_pb_DeleteSnapshotResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeleteSnapshotResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_RestoreSnapshotRequest_descriptor =
       getDescriptor().getMessageTypes().get(76);
     internal_static_hbase_pb_RestoreSnapshotRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RestoreSnapshotRequest_descriptor,
         new java.lang.String[] { "Snapshot", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_RestoreSnapshotResponse_descriptor =
       getDescriptor().getMessageTypes().get(77);
     internal_static_hbase_pb_RestoreSnapshotResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RestoreSnapshotResponse_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_IsSnapshotDoneRequest_descriptor =
       getDescriptor().getMessageTypes().get(78);
     internal_static_hbase_pb_IsSnapshotDoneRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsSnapshotDoneRequest_descriptor,
         new java.lang.String[] { "Snapshot", });
     internal_static_hbase_pb_IsSnapshotDoneResponse_descriptor =
       getDescriptor().getMessageTypes().get(79);
     internal_static_hbase_pb_IsSnapshotDoneResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsSnapshotDoneResponse_descriptor,
         new java.lang.String[] { "Done", "Snapshot", });
     internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_descriptor =
       getDescriptor().getMessageTypes().get(80);
     internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsRestoreSnapshotDoneRequest_descriptor,
         new java.lang.String[] { "Snapshot", });
     internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_descriptor =
       getDescriptor().getMessageTypes().get(81);
     internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsRestoreSnapshotDoneResponse_descriptor,
         new java.lang.String[] { "Done", });
     internal_static_hbase_pb_GetSchemaAlterStatusRequest_descriptor =
       getDescriptor().getMessageTypes().get(82);
     internal_static_hbase_pb_GetSchemaAlterStatusRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetSchemaAlterStatusRequest_descriptor,
         new java.lang.String[] { "TableName", });
     internal_static_hbase_pb_GetSchemaAlterStatusResponse_descriptor =
       getDescriptor().getMessageTypes().get(83);
     internal_static_hbase_pb_GetSchemaAlterStatusResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetSchemaAlterStatusResponse_descriptor,
         new java.lang.String[] { "YetToUpdateRegions", "TotalRegions", });
     internal_static_hbase_pb_GetTableDescriptorsRequest_descriptor =
       getDescriptor().getMessageTypes().get(84);
     internal_static_hbase_pb_GetTableDescriptorsRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetTableDescriptorsRequest_descriptor,
         new java.lang.String[] { "TableNames", "Regex", "IncludeSysTables", "Namespace", });
     internal_static_hbase_pb_GetTableDescriptorsResponse_descriptor =
       getDescriptor().getMessageTypes().get(85);
     internal_static_hbase_pb_GetTableDescriptorsResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetTableDescriptorsResponse_descriptor,
         new java.lang.String[] { "TableSchema", });
     internal_static_hbase_pb_GetTableNamesRequest_descriptor =
       getDescriptor().getMessageTypes().get(86);
     internal_static_hbase_pb_GetTableNamesRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetTableNamesRequest_descriptor,
         new java.lang.String[] { "Regex", "IncludeSysTables", "Namespace", });
     internal_static_hbase_pb_GetTableNamesResponse_descriptor =
       getDescriptor().getMessageTypes().get(87);
     internal_static_hbase_pb_GetTableNamesResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetTableNamesResponse_descriptor,
         new java.lang.String[] { "TableNames", });
     internal_static_hbase_pb_GetTableStateRequest_descriptor =
       getDescriptor().getMessageTypes().get(88);
     internal_static_hbase_pb_GetTableStateRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetTableStateRequest_descriptor,
         new java.lang.String[] { "TableName", });
     internal_static_hbase_pb_GetTableStateResponse_descriptor =
       getDescriptor().getMessageTypes().get(89);
     internal_static_hbase_pb_GetTableStateResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetTableStateResponse_descriptor,
         new java.lang.String[] { "TableState", });
     internal_static_hbase_pb_GetClusterStatusRequest_descriptor =
       getDescriptor().getMessageTypes().get(90);
     internal_static_hbase_pb_GetClusterStatusRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetClusterStatusRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_GetClusterStatusResponse_descriptor =
       getDescriptor().getMessageTypes().get(91);
     internal_static_hbase_pb_GetClusterStatusResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetClusterStatusResponse_descriptor,
         new java.lang.String[] { "ClusterStatus", });
     internal_static_hbase_pb_IsMasterRunningRequest_descriptor =
       getDescriptor().getMessageTypes().get(92);
     internal_static_hbase_pb_IsMasterRunningRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsMasterRunningRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_IsMasterRunningResponse_descriptor =
       getDescriptor().getMessageTypes().get(93);
     internal_static_hbase_pb_IsMasterRunningResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsMasterRunningResponse_descriptor,
         new java.lang.String[] { "IsMasterRunning", });
     internal_static_hbase_pb_ExecProcedureRequest_descriptor =
       getDescriptor().getMessageTypes().get(94);
     internal_static_hbase_pb_ExecProcedureRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ExecProcedureRequest_descriptor,
         new java.lang.String[] { "Procedure", });
     internal_static_hbase_pb_ExecProcedureResponse_descriptor =
       getDescriptor().getMessageTypes().get(95);
     internal_static_hbase_pb_ExecProcedureResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ExecProcedureResponse_descriptor,
         new java.lang.String[] { "ExpectedTimeout", "ReturnData", });
     internal_static_hbase_pb_IsProcedureDoneRequest_descriptor =
       getDescriptor().getMessageTypes().get(96);
     internal_static_hbase_pb_IsProcedureDoneRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsProcedureDoneRequest_descriptor,
         new java.lang.String[] { "Procedure", });
     internal_static_hbase_pb_IsProcedureDoneResponse_descriptor =
       getDescriptor().getMessageTypes().get(97);
     internal_static_hbase_pb_IsProcedureDoneResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_IsProcedureDoneResponse_descriptor,
         new java.lang.String[] { "Done", "Snapshot", });
     internal_static_hbase_pb_GetProcedureResultRequest_descriptor =
       getDescriptor().getMessageTypes().get(98);
     internal_static_hbase_pb_GetProcedureResultRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetProcedureResultRequest_descriptor,
         new java.lang.String[] { "ProcId", });
     internal_static_hbase_pb_GetProcedureResultResponse_descriptor =
       getDescriptor().getMessageTypes().get(99);
     internal_static_hbase_pb_GetProcedureResultResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetProcedureResultResponse_descriptor,
         new java.lang.String[] { "State", "StartTime", "LastUpdate", "Result", "Exception", });
     internal_static_hbase_pb_AbortProcedureRequest_descriptor =
       getDescriptor().getMessageTypes().get(100);
     internal_static_hbase_pb_AbortProcedureRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AbortProcedureRequest_descriptor,
         new java.lang.String[] { "ProcId", "MayInterruptIfRunning", });
     internal_static_hbase_pb_AbortProcedureResponse_descriptor =
       getDescriptor().getMessageTypes().get(101);
     internal_static_hbase_pb_AbortProcedureResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_AbortProcedureResponse_descriptor,
         new java.lang.String[] { "IsProcedureAborted", });
     internal_static_hbase_pb_ListProceduresRequest_descriptor =
       getDescriptor().getMessageTypes().get(102);
     internal_static_hbase_pb_ListProceduresRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListProceduresRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_ListProceduresResponse_descriptor =
       getDescriptor().getMessageTypes().get(103);
     internal_static_hbase_pb_ListProceduresResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ListProceduresResponse_descriptor,
         new java.lang.String[] { "Procedure", });
     internal_static_hbase_pb_SetQuotaRequest_descriptor =
       getDescriptor().getMessageTypes().get(104);
     internal_static_hbase_pb_SetQuotaRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetQuotaRequest_descriptor,
         new java.lang.String[] { "UserName", "UserGroup", "Namespace", "TableName", "RemoveAll", "BypassGlobals", "Throttle", });
     internal_static_hbase_pb_SetQuotaResponse_descriptor =
       getDescriptor().getMessageTypes().get(105);
     internal_static_hbase_pb_SetQuotaResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SetQuotaResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_MajorCompactionTimestampRequest_descriptor =
       getDescriptor().getMessageTypes().get(106);
     internal_static_hbase_pb_MajorCompactionTimestampRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MajorCompactionTimestampRequest_descriptor,
         new java.lang.String[] { "TableName", });
     internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_descriptor =
       getDescriptor().getMessageTypes().get(107);
     internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MajorCompactionTimestampForRegionRequest_descriptor,
         new java.lang.String[] { "Region", });
     internal_static_hbase_pb_MajorCompactionTimestampResponse_descriptor =
       getDescriptor().getMessageTypes().get(108);
     internal_static_hbase_pb_MajorCompactionTimestampResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MajorCompactionTimestampResponse_descriptor,
         new java.lang.String[] { "CompactionTimestamp", });
     internal_static_hbase_pb_SecurityCapabilitiesRequest_descriptor =
       getDescriptor().getMessageTypes().get(109);
     internal_static_hbase_pb_SecurityCapabilitiesRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SecurityCapabilitiesRequest_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor =
       getDescriptor().getMessageTypes().get(110);
     internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor,
         new java.lang.String[] { "Capabilities", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
index 40e4611..91989d7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
@@ -6,19 +6,19 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ProcedureProtos {
   private ProcedureProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * Protobuf enum {@code hbase.pb.ProcedureState}
    */
   public enum ProcedureState
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * 
      * Procedure in construction, not yet added to the executor
@@ -143,27 +143,27 @@ public final class ProcedureProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ProcedureState> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ProcedureState findValueByNumber(int number) {
               return ProcedureState.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -171,7 +171,7 @@ public final class ProcedureProtos {
     private static final ProcedureState[] VALUES = values();
 
     public static ProcedureState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -190,7 +190,7 @@ public final class ProcedureProtos {
 
   public interface ProcedureOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Procedure)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -215,7 +215,7 @@ public final class ProcedureProtos {
      *
      * required string class_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClassNameBytes();
 
     /**
@@ -264,7 +264,7 @@ public final class ProcedureProtos {
     /**
      * optional string owner = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getOwnerBytes();
 
     /**
@@ -367,7 +367,7 @@ public final class ProcedureProtos {
      *
      * optional bytes result = 11;
      */
-    com.google.protobuf.ByteString getResult();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getResult();
 
     /**
      * 
@@ -384,7 +384,7 @@ public final class ProcedureProtos {
      *
      * optional bytes state_data = 12;
      */
-    com.google.protobuf.ByteString getStateData();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStateData();
 
     /**
      * 
@@ -421,11 +421,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.Procedure}
    */
   public  static final class Procedure extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Procedure)
       ProcedureOrBuilder {
     // Use Procedure.newBuilder() to construct.
-    private Procedure(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Procedure(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Procedure() {
@@ -438,25 +438,25 @@ public final class ProcedureProtos {
       stackId_ = java.util.Collections.emptyList();
       lastUpdate_ = 0L;
       timeout_ = 0;
-      result_ = com.google.protobuf.ByteString.EMPTY;
-      stateData_ = com.google.protobuf.ByteString.EMPTY;
+      result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      stateData_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       nonceGroup_ = 0L;
       nonce_ = 0L;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Procedure(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -473,7 +473,7 @@ public final class ProcedureProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               className_ = bs;
               break;
@@ -494,7 +494,7 @@ public final class ProcedureProtos {
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000010;
               owner_ = bs;
               break;
@@ -576,10 +576,10 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
@@ -589,12 +589,12 @@ public final class ProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_Procedure_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_Procedure_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -626,8 +626,8 @@ public final class ProcedureProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           className_ = s;
@@ -642,17 +642,17 @@ public final class ProcedureProtos {
      *
      * required string class_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClassNameBytes() {
       java.lang.Object ref = className_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         className_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -725,8 +725,8 @@ public final class ProcedureProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           owner_ = s;
@@ -737,17 +737,17 @@ public final class ProcedureProtos {
     /**
      * optional string owner = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getOwnerBytes() {
       java.lang.Object ref = owner_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         owner_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -873,7 +873,7 @@ public final class ProcedureProtos {
     }
 
     public static final int RESULT_FIELD_NUMBER = 11;
-    private com.google.protobuf.ByteString result_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString result_;
     /**
      * 
      * opaque (user) result structure
@@ -891,12 +891,12 @@ public final class ProcedureProtos {
      *
      * optional bytes result = 11;
      */
-    public com.google.protobuf.ByteString getResult() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getResult() {
       return result_;
     }
 
     public static final int STATE_DATA_FIELD_NUMBER = 12;
-    private com.google.protobuf.ByteString stateData_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stateData_;
     /**
      * 
      * opaque (user) procedure internal-state
@@ -914,7 +914,7 @@ public final class ProcedureProtos {
      *
      * optional bytes state_data = 12;
      */
-    public com.google.protobuf.ByteString getStateData() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStateData() {
       return stateData_;
     }
 
@@ -986,10 +986,10 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, className_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, className_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt64(2, parentId_);
@@ -1001,7 +1001,7 @@ public final class ProcedureProtos {
         output.writeUInt64(4, startTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, owner_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, owner_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeEnum(6, state_);
@@ -1039,62 +1039,62 @@ public final class ProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, className_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, className_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, parentId_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, procId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, startTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, owner_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, owner_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(6, state_);
       }
       {
         int dataSize = 0;
         for (int i = 0; i < stackId_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt32SizeNoTag(stackId_.get(i));
         }
         size += dataSize;
         size += 1 * getStackIdList().size();
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(8, lastUpdate_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(9, timeout_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(10, getException());
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(11, result_);
       }
       if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(12, stateData_);
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(13, nonceGroup_);
       }
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(14, nonce_);
       }
       size += unknownFields.getSerializedSize();
@@ -1197,17 +1197,17 @@ public final class ProcedureProtos {
       }
       if (hasParentId()) {
         hash = (37 * hash) + PARENT_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getParentId());
       }
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       if (hasStartTime()) {
         hash = (37 * hash) + START_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getStartTime());
       }
       if (hasOwner()) {
@@ -1224,7 +1224,7 @@ public final class ProcedureProtos {
       }
       if (hasLastUpdate()) {
         hash = (37 * hash) + LAST_UPDATE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastUpdate());
       }
       if (hasTimeout()) {
@@ -1245,12 +1245,12 @@ public final class ProcedureProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -1259,61 +1259,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1331,7 +1331,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1344,15 +1344,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.Procedure}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Procedure)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_Procedure_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_Procedure_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1365,12 +1365,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getExceptionFieldBuilder();
         }
@@ -1401,9 +1401,9 @@ public final class ProcedureProtos {
           exceptionBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000200);
-        result_ = com.google.protobuf.ByteString.EMPTY;
+        result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000400);
-        stateData_ = com.google.protobuf.ByteString.EMPTY;
+        stateData_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000800);
         nonceGroup_ = 0L;
         bitField0_ = (bitField0_ & ~0x00001000);
@@ -1412,7 +1412,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_Procedure_descriptor;
       }
@@ -1503,29 +1503,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure)other);
         } else {
@@ -1614,13 +1614,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1653,8 +1653,8 @@ public final class ProcedureProtos {
       public java.lang.String getClassName() {
         java.lang.Object ref = className_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             className_ = s;
@@ -1671,17 +1671,17 @@ public final class ProcedureProtos {
        *
        * required string class_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getClassNameBytes() {
         java.lang.Object ref = className_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           className_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1722,7 +1722,7 @@ public final class ProcedureProtos {
        * required string class_name = 1;
        */
       public Builder setClassNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1857,8 +1857,8 @@ public final class ProcedureProtos {
       public java.lang.String getOwner() {
         java.lang.Object ref = owner_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             owner_ = s;
@@ -1871,17 +1871,17 @@ public final class ProcedureProtos {
       /**
        * optional string owner = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getOwnerBytes() {
         java.lang.Object ref = owner_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           owner_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1910,7 +1910,7 @@ public final class ProcedureProtos {
        * optional string owner = 5;
        */
       public Builder setOwnerBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2047,7 +2047,7 @@ public final class ProcedureProtos {
       public Builder addAllStackId(
           java.lang.Iterable values) {
         ensureStackIdIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, stackId_);
         onChanged();
         return this;
@@ -2131,7 +2131,7 @@ public final class ProcedureProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage exception_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder> exceptionBuilder_;
       /**
        * 
@@ -2270,11 +2270,11 @@ public final class ProcedureProtos {
        *
        * optional .hbase.pb.ForeignExceptionMessage exception = 10;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder> 
           getExceptionFieldBuilder() {
         if (exceptionBuilder_ == null) {
-          exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          exceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder>(
                   getException(),
                   getParentForChildren(),
@@ -2284,7 +2284,7 @@ public final class ProcedureProtos {
         return exceptionBuilder_;
       }
 
-      private com.google.protobuf.ByteString result_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        * opaque (user) result structure
@@ -2302,7 +2302,7 @@ public final class ProcedureProtos {
        *
        * optional bytes result = 11;
        */
-      public com.google.protobuf.ByteString getResult() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getResult() {
         return result_;
       }
       /**
@@ -2312,7 +2312,7 @@ public final class ProcedureProtos {
        *
        * optional bytes result = 11;
        */
-      public Builder setResult(com.google.protobuf.ByteString value) {
+      public Builder setResult(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2335,7 +2335,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString stateData_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString stateData_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        * opaque (user) procedure internal-state
@@ -2353,7 +2353,7 @@ public final class ProcedureProtos {
        *
        * optional bytes state_data = 12;
        */
-      public com.google.protobuf.ByteString getStateData() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getStateData() {
         return stateData_;
       }
       /**
@@ -2363,7 +2363,7 @@ public final class ProcedureProtos {
        *
        * optional bytes state_data = 12;
        */
-      public Builder setStateData(com.google.protobuf.ByteString value) {
+      public Builder setStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2466,12 +2466,12 @@ public final class ProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2489,22 +2489,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Procedure parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Procedure(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2516,7 +2516,7 @@ public final class ProcedureProtos {
 
   public interface SequentialProcedureDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SequentialProcedureData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bool executed = 1;
@@ -2536,11 +2536,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.SequentialProcedureData}
    */
   public  static final class SequentialProcedureData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SequentialProcedureData)
       SequentialProcedureDataOrBuilder {
     // Use SequentialProcedureData.newBuilder() to construct.
-    private SequentialProcedureData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SequentialProcedureData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SequentialProcedureData() {
@@ -2548,18 +2548,18 @@ public final class ProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SequentialProcedureData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2582,22 +2582,22 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_SequentialProcedureData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_SequentialProcedureData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2634,7 +2634,7 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, executed_);
@@ -2648,7 +2648,7 @@ public final class ProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, executed_);
       }
       size += unknownFields.getSerializedSize();
@@ -2686,7 +2686,7 @@ public final class ProcedureProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasExecuted()) {
         hash = (37 * hash) + EXECUTED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getExecuted());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -2695,61 +2695,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2767,7 +2767,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2780,15 +2780,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.SequentialProcedureData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SequentialProcedureData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_SequentialProcedureData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_SequentialProcedureData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2801,12 +2801,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -2817,7 +2817,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_SequentialProcedureData_descriptor;
       }
@@ -2851,29 +2851,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData)other);
         } else {
@@ -2900,13 +2900,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2950,12 +2950,12 @@ public final class ProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2973,22 +2973,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SequentialProcedureData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SequentialProcedureData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3000,7 +3000,7 @@ public final class ProcedureProtos {
 
   public interface StateMachineProcedureDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StateMachineProcedureData)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated uint32 state = 1;
@@ -3024,11 +3024,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.StateMachineProcedureData}
    */
   public  static final class StateMachineProcedureData extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StateMachineProcedureData)
       StateMachineProcedureDataOrBuilder {
     // Use StateMachineProcedureData.newBuilder() to construct.
-    private StateMachineProcedureData(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StateMachineProcedureData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StateMachineProcedureData() {
@@ -3036,18 +3036,18 @@ public final class ProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StateMachineProcedureData(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3086,10 +3086,10 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -3099,12 +3099,12 @@ public final class ProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_StateMachineProcedureData_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_StateMachineProcedureData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3143,7 +3143,7 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < state_.size(); i++) {
         output.writeUInt32(1, state_.get(i));
@@ -3159,7 +3159,7 @@ public final class ProcedureProtos {
       {
         int dataSize = 0;
         for (int i = 0; i < state_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt32SizeNoTag(state_.get(i));
         }
         size += dataSize;
@@ -3205,61 +3205,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3277,7 +3277,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3290,15 +3290,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.StateMachineProcedureData}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StateMachineProcedureData)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureDataOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_StateMachineProcedureData_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_StateMachineProcedureData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3311,12 +3311,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3327,7 +3327,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_StateMachineProcedureData_descriptor;
       }
@@ -3360,29 +3360,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData)other);
         } else {
@@ -3413,13 +3413,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3482,7 +3482,7 @@ public final class ProcedureProtos {
       public Builder addAllState(
           java.lang.Iterable values) {
         ensureStateIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, state_);
         onChanged();
         return this;
@@ -3497,12 +3497,12 @@ public final class ProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3520,22 +3520,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StateMachineProcedureData parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StateMachineProcedureData(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3547,7 +3547,7 @@ public final class ProcedureProtos {
 
   public interface ProcedureWALHeaderOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ProcedureWALHeader)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint32 version = 1;
@@ -3594,11 +3594,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.ProcedureWALHeader}
    */
   public  static final class ProcedureWALHeader extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ProcedureWALHeader)
       ProcedureWALHeaderOrBuilder {
     // Use ProcedureWALHeader.newBuilder() to construct.
-    private ProcedureWALHeader(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ProcedureWALHeader(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ProcedureWALHeader() {
@@ -3609,18 +3609,18 @@ public final class ProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ProcedureWALHeader(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3658,22 +3658,22 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALHeader_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALHeader_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3767,7 +3767,7 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, version_);
@@ -3790,19 +3790,19 @@ public final class ProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, version_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, type_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, logId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, minProcId_);
       }
       size += unknownFields.getSerializedSize();
@@ -3863,12 +3863,12 @@ public final class ProcedureProtos {
       }
       if (hasLogId()) {
         hash = (37 * hash) + LOG_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLogId());
       }
       if (hasMinProcId()) {
         hash = (37 * hash) + MIN_PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getMinProcId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -3877,61 +3877,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3949,7 +3949,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3962,15 +3962,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.ProcedureWALHeader}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ProcedureWALHeader)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeaderOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALHeader_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALHeader_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3983,12 +3983,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -4005,7 +4005,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALHeader_descriptor;
       }
@@ -4051,29 +4051,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader)other);
         } else {
@@ -4118,13 +4118,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4264,12 +4264,12 @@ public final class ProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4287,22 +4287,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ProcedureWALHeader parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ProcedureWALHeader(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4314,7 +4314,7 @@ public final class ProcedureProtos {
 
   public interface ProcedureWALTrailerOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ProcedureWALTrailer)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required uint32 version = 1;
@@ -4343,11 +4343,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.ProcedureWALTrailer}
    */
   public  static final class ProcedureWALTrailer extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ProcedureWALTrailer)
       ProcedureWALTrailerOrBuilder {
     // Use ProcedureWALTrailer.newBuilder() to construct.
-    private ProcedureWALTrailer(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ProcedureWALTrailer(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ProcedureWALTrailer() {
@@ -4356,18 +4356,18 @@ public final class ProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ProcedureWALTrailer(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4395,22 +4395,22 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALTrailer_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALTrailer_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4466,7 +4466,7 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, version_);
@@ -4483,11 +4483,11 @@ public final class ProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, version_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, trackerPos_);
       }
       size += unknownFields.getSerializedSize();
@@ -4534,7 +4534,7 @@ public final class ProcedureProtos {
       }
       if (hasTrackerPos()) {
         hash = (37 * hash) + TRACKER_POS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTrackerPos());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -4543,61 +4543,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4615,7 +4615,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4628,15 +4628,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.ProcedureWALTrailer}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ProcedureWALTrailer)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailerOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALTrailer_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALTrailer_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4649,12 +4649,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -4667,7 +4667,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALTrailer_descriptor;
       }
@@ -4705,29 +4705,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer)other);
         } else {
@@ -4760,13 +4760,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4842,12 +4842,12 @@ public final class ProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4865,22 +4865,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ProcedureWALTrailer parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ProcedureWALTrailer(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4892,7 +4892,7 @@ public final class ProcedureProtos {
 
   public interface ProcedureStoreTrackerOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ProcedureStoreTracker)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * repeated .hbase.pb.ProcedureStoreTracker.TrackerNode node = 1;
@@ -4922,11 +4922,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.ProcedureStoreTracker}
    */
   public  static final class ProcedureStoreTracker extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ProcedureStoreTracker)
       ProcedureStoreTrackerOrBuilder {
     // Use ProcedureStoreTracker.newBuilder() to construct.
-    private ProcedureStoreTracker(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ProcedureStoreTracker(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ProcedureStoreTracker() {
@@ -4934,18 +4934,18 @@ public final class ProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ProcedureStoreTracker(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4972,10 +4972,10 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -4985,12 +4985,12 @@ public final class ProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4999,7 +4999,7 @@ public final class ProcedureProtos {
 
     public interface TrackerNodeOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.ProcedureStoreTracker.TrackerNode)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required uint64 start_id = 1;
@@ -5040,11 +5040,11 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.ProcedureStoreTracker.TrackerNode}
      */
     public  static final class TrackerNode extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.ProcedureStoreTracker.TrackerNode)
         TrackerNodeOrBuilder {
       // Use TrackerNode.newBuilder() to construct.
-      private TrackerNode(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private TrackerNode(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private TrackerNode() {
@@ -5054,18 +5054,18 @@ public final class ProcedureProtos {
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private TrackerNode(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -5130,10 +5130,10 @@ public final class ProcedureProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -5146,12 +5146,12 @@ public final class ProcedureProtos {
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5232,7 +5232,7 @@ public final class ProcedureProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeUInt64(1, startId_);
@@ -5252,13 +5252,13 @@ public final class ProcedureProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt64Size(1, startId_);
         }
         {
           int dataSize = 0;
           for (int i = 0; i < updated_.size(); i++) {
-            dataSize += com.google.protobuf.CodedOutputStream
+            dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeUInt64SizeNoTag(updated_.get(i));
           }
           size += dataSize;
@@ -5267,7 +5267,7 @@ public final class ProcedureProtos {
         {
           int dataSize = 0;
           for (int i = 0; i < deleted_.size(); i++) {
-            dataSize += com.google.protobuf.CodedOutputStream
+            dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
               .computeUInt64SizeNoTag(deleted_.get(i));
           }
           size += dataSize;
@@ -5312,7 +5312,7 @@ public final class ProcedureProtos {
         hash = (19 * hash) + getDescriptorForType().hashCode();
         if (hasStartId()) {
           hash = (37 * hash) + START_ID_FIELD_NUMBER;
-          hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+          hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
               getStartId());
         }
         if (getUpdatedCount() > 0) {
@@ -5329,61 +5329,61 @@ public final class ProcedureProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -5401,7 +5401,7 @@ public final class ProcedureProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -5409,15 +5409,15 @@ public final class ProcedureProtos {
        * Protobuf type {@code hbase.pb.ProcedureStoreTracker.TrackerNode}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.ProcedureStoreTracker.TrackerNode)
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNodeOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -5430,12 +5430,12 @@ public final class ProcedureProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
           }
         }
@@ -5450,7 +5450,7 @@ public final class ProcedureProtos {
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_descriptor;
         }
@@ -5494,29 +5494,29 @@ public final class ProcedureProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode)other);
           } else {
@@ -5563,13 +5563,13 @@ public final class ProcedureProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -5664,7 +5664,7 @@ public final class ProcedureProtos {
         public Builder addAllUpdated(
             java.lang.Iterable values) {
           ensureUpdatedIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, updated_);
           onChanged();
           return this;
@@ -5730,7 +5730,7 @@ public final class ProcedureProtos {
         public Builder addAllDeleted(
             java.lang.Iterable values) {
           ensureDeletedIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, deleted_);
           onChanged();
           return this;
@@ -5745,12 +5745,12 @@ public final class ProcedureProtos {
           return this;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -5768,22 +5768,22 @@ public final class ProcedureProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public TrackerNode parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new TrackerNode(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -5844,7 +5844,7 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < node_.size(); i++) {
         output.writeMessage(1, node_.get(i));
@@ -5858,7 +5858,7 @@ public final class ProcedureProtos {
 
       size = 0;
       for (int i = 0; i < node_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, node_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -5901,61 +5901,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5973,7 +5973,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5981,15 +5981,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.ProcedureStoreTracker}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ProcedureStoreTracker)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTrackerOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6002,12 +6002,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getNodeFieldBuilder();
         }
@@ -6023,7 +6023,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureStoreTracker_descriptor;
       }
@@ -6060,29 +6060,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker)other);
         } else {
@@ -6112,7 +6112,7 @@ public final class ProcedureProtos {
               node_ = other.node_;
               bitField0_ = (bitField0_ & ~0x00000001);
               nodeBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getNodeFieldBuilder() : null;
             } else {
               nodeBuilder_.addAllMessages(other.node_);
@@ -6134,13 +6134,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6161,7 +6161,7 @@ public final class ProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNodeOrBuilder> nodeBuilder_;
 
       /**
@@ -6293,7 +6293,7 @@ public final class ProcedureProtos {
           java.lang.Iterable values) {
         if (nodeBuilder_ == null) {
           ensureNodeIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, node_);
           onChanged();
         } else {
@@ -6377,11 +6377,11 @@ public final class ProcedureProtos {
            getNodeBuilderList() {
         return getNodeFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNodeOrBuilder> 
           getNodeFieldBuilder() {
         if (nodeBuilder_ == null) {
-          nodeBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          nodeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNodeOrBuilder>(
                   node_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -6392,12 +6392,12 @@ public final class ProcedureProtos {
         return nodeBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6415,22 +6415,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ProcedureStoreTracker parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ProcedureStoreTracker(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6442,7 +6442,7 @@ public final class ProcedureProtos {
 
   public interface ProcedureWALEntryOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ProcedureWALEntry)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ProcedureWALEntry.Type type = 1;
@@ -6503,11 +6503,11 @@ public final class ProcedureProtos {
    * Protobuf type {@code hbase.pb.ProcedureWALEntry}
    */
   public  static final class ProcedureWALEntry extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ProcedureWALEntry)
       ProcedureWALEntryOrBuilder {
     // Use ProcedureWALEntry.newBuilder() to construct.
-    private ProcedureWALEntry(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ProcedureWALEntry(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ProcedureWALEntry() {
@@ -6518,18 +6518,18 @@ public final class ProcedureProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ProcedureWALEntry(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6593,10 +6593,10 @@ public final class ProcedureProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -6609,12 +6609,12 @@ public final class ProcedureProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALEntry_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALEntry_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6625,7 +6625,7 @@ public final class ProcedureProtos {
      * Protobuf enum {@code hbase.pb.ProcedureWALEntry.Type}
      */
     public enum Type
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * PROCEDURE_WAL_EOF = 1;
        */
@@ -6702,27 +6702,27 @@ public final class ProcedureProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Type> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Type findValueByNumber(int number) {
                 return Type.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry.getDescriptor().getEnumTypes().get(0);
       }
@@ -6730,7 +6730,7 @@ public final class ProcedureProtos {
       private static final Type[] VALUES = values();
 
       public static Type valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -6856,7 +6856,7 @@ public final class ProcedureProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, type_);
@@ -6879,21 +6879,21 @@ public final class ProcedureProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, type_);
       }
       for (int i = 0; i < procedure_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, procedure_.get(i));
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, procId_);
       }
       {
         int dataSize = 0;
         for (int i = 0; i < childId_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt64SizeNoTag(childId_.get(i));
         }
         size += dataSize;
@@ -6950,7 +6950,7 @@ public final class ProcedureProtos {
       }
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
       if (getChildIdCount() > 0) {
@@ -6963,61 +6963,61 @@ public final class ProcedureProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7035,7 +7035,7 @@ public final class ProcedureProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7043,15 +7043,15 @@ public final class ProcedureProtos {
      * Protobuf type {@code hbase.pb.ProcedureWALEntry}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ProcedureWALEntry)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntryOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALEntry_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALEntry_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7064,12 +7064,12 @@ public final class ProcedureProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getProcedureFieldBuilder();
         }
@@ -7091,7 +7091,7 @@ public final class ProcedureProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.internal_static_hbase_pb_ProcedureWALEntry_descriptor;
       }
@@ -7143,29 +7143,29 @@ public final class ProcedureProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry)other);
         } else {
@@ -7198,7 +7198,7 @@ public final class ProcedureProtos {
               procedure_ = other.procedure_;
               bitField0_ = (bitField0_ & ~0x00000002);
               procedureBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getProcedureFieldBuilder() : null;
             } else {
               procedureBuilder_.addAllMessages(other.procedure_);
@@ -7236,13 +7236,13 @@ public final class ProcedureProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7299,7 +7299,7 @@ public final class ProcedureProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> procedureBuilder_;
 
       /**
@@ -7431,7 +7431,7 @@ public final class ProcedureProtos {
           java.lang.Iterable values) {
         if (procedureBuilder_ == null) {
           ensureProcedureIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, procedure_);
           onChanged();
         } else {
@@ -7515,11 +7515,11 @@ public final class ProcedureProtos {
            getProcedureBuilderList() {
         return getProcedureFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> 
           getProcedureFieldBuilder() {
         if (procedureBuilder_ == null) {
-          procedureBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          procedureBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder>(
                   procedure_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -7613,7 +7613,7 @@ public final class ProcedureProtos {
       public Builder addAllChildId(
           java.lang.Iterable values) {
         ensureChildIdIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, childId_);
         onChanged();
         return this;
@@ -7628,12 +7628,12 @@ public final class ProcedureProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7651,22 +7651,22 @@ public final class ProcedureProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ProcedureWALEntry parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ProcedureWALEntry(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7676,52 +7676,52 @@ public final class ProcedureProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Procedure_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Procedure_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SequentialProcedureData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SequentialProcedureData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StateMachineProcedureData_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StateMachineProcedureData_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureWALHeader_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ProcedureWALHeader_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureWALTrailer_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ProcedureWALTrailer_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureStoreTracker_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ProcedureStoreTracker_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ProcedureWALEntry_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ProcedureWALEntry_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -7758,65 +7758,65 @@ public final class ProcedureProtos {
       "g.apache.hadoop.hbase.shaded.protobuf.ge" +
       "neratedB\017ProcedureProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_Procedure_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_Procedure_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Procedure_descriptor,
         new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_SequentialProcedureData_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_SequentialProcedureData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SequentialProcedureData_descriptor,
         new java.lang.String[] { "Executed", });
     internal_static_hbase_pb_StateMachineProcedureData_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_StateMachineProcedureData_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StateMachineProcedureData_descriptor,
         new java.lang.String[] { "State", });
     internal_static_hbase_pb_ProcedureWALHeader_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_ProcedureWALHeader_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ProcedureWALHeader_descriptor,
         new java.lang.String[] { "Version", "Type", "LogId", "MinProcId", });
     internal_static_hbase_pb_ProcedureWALTrailer_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_ProcedureWALTrailer_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ProcedureWALTrailer_descriptor,
         new java.lang.String[] { "Version", "TrackerPos", });
     internal_static_hbase_pb_ProcedureStoreTracker_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_ProcedureStoreTracker_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ProcedureStoreTracker_descriptor,
         new java.lang.String[] { "Node", });
     internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_descriptor =
       internal_static_hbase_pb_ProcedureStoreTracker_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ProcedureStoreTracker_TrackerNode_descriptor,
         new java.lang.String[] { "StartId", "Updated", "Deleted", });
     internal_static_hbase_pb_ProcedureWALEntry_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_ProcedureWALEntry_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ProcedureWALEntry_descriptor,
         new java.lang.String[] { "Type", "Procedure", "ProcId", "ChildId", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index d31e338..d14336a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -6,19 +6,19 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class QuotaProtos {
   private QuotaProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * Protobuf enum {@code hbase.pb.QuotaScope}
    */
   public enum QuotaScope
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * CLUSTER = 1;
      */
@@ -59,27 +59,27 @@ public final class QuotaProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         QuotaScope> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public QuotaScope findValueByNumber(int number) {
               return QuotaScope.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -87,7 +87,7 @@ public final class QuotaProtos {
     private static final QuotaScope[] VALUES = values();
 
     public static QuotaScope valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -108,7 +108,7 @@ public final class QuotaProtos {
    * Protobuf enum {@code hbase.pb.ThrottleType}
    */
   public enum ThrottleType
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * REQUEST_NUMBER = 1;
      */
@@ -185,27 +185,27 @@ public final class QuotaProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ThrottleType> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ThrottleType findValueByNumber(int number) {
               return ThrottleType.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor().getEnumTypes().get(1);
     }
@@ -213,7 +213,7 @@ public final class QuotaProtos {
     private static final ThrottleType[] VALUES = values();
 
     public static ThrottleType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -234,7 +234,7 @@ public final class QuotaProtos {
    * Protobuf enum {@code hbase.pb.QuotaType}
    */
   public enum QuotaType
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * THROTTLE = 1;
      */
@@ -266,27 +266,27 @@ public final class QuotaProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         QuotaType> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public QuotaType findValueByNumber(int number) {
               return QuotaType.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor().getEnumTypes().get(2);
     }
@@ -294,7 +294,7 @@ public final class QuotaProtos {
     private static final QuotaType[] VALUES = values();
 
     public static QuotaType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -313,7 +313,7 @@ public final class QuotaProtos {
 
   public interface TimedQuotaOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TimedQuota)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TimeUnit time_unit = 1;
@@ -355,11 +355,11 @@ public final class QuotaProtos {
    * Protobuf type {@code hbase.pb.TimedQuota}
    */
   public  static final class TimedQuota extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TimedQuota)
       TimedQuotaOrBuilder {
     // Use TimedQuota.newBuilder() to construct.
-    private TimedQuota(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TimedQuota(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TimedQuota() {
@@ -370,18 +370,18 @@ public final class QuotaProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TimedQuota(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -431,22 +431,22 @@ public final class QuotaProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_TimedQuota_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_TimedQuota_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -530,7 +530,7 @@ public final class QuotaProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, timeUnit_);
@@ -553,19 +553,19 @@ public final class QuotaProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, timeUnit_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, softLimit_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeFloatSize(3, share_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(4, scope_);
       }
       size += unknownFields.getSerializedSize();
@@ -622,7 +622,7 @@ public final class QuotaProtos {
       }
       if (hasSoftLimit()) {
         hash = (37 * hash) + SOFT_LIMIT_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getSoftLimit());
       }
       if (hasShare()) {
@@ -640,61 +640,61 @@ public final class QuotaProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -712,7 +712,7 @@ public final class QuotaProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -720,15 +720,15 @@ public final class QuotaProtos {
      * Protobuf type {@code hbase.pb.TimedQuota}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TimedQuota)
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_TimedQuota_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_TimedQuota_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -741,12 +741,12 @@ public final class QuotaProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -763,7 +763,7 @@ public final class QuotaProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_TimedQuota_descriptor;
       }
@@ -809,29 +809,29 @@ public final class QuotaProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota)other);
         } else {
@@ -867,13 +867,13 @@ public final class QuotaProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1021,12 +1021,12 @@ public final class QuotaProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1044,22 +1044,22 @@ public final class QuotaProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TimedQuota parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TimedQuota(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1071,7 +1071,7 @@ public final class QuotaProtos {
 
   public interface ThrottleOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Throttle)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.TimedQuota req_num = 1;
@@ -1155,29 +1155,29 @@ public final class QuotaProtos {
    * Protobuf type {@code hbase.pb.Throttle}
    */
   public  static final class Throttle extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Throttle)
       ThrottleOrBuilder {
     // Use Throttle.newBuilder() to construct.
-    private Throttle(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Throttle(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Throttle() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Throttle(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1273,22 +1273,22 @@ public final class QuotaProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Throttle_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Throttle_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1468,7 +1468,7 @@ public final class QuotaProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getReqNum());
@@ -1497,27 +1497,27 @@ public final class QuotaProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getReqNum());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getReqSize());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getWriteNum());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, getWriteSize());
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getReadNum());
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, getReadSize());
       }
       size += unknownFields.getSerializedSize();
@@ -1608,61 +1608,61 @@ public final class QuotaProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1680,7 +1680,7 @@ public final class QuotaProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1688,15 +1688,15 @@ public final class QuotaProtos {
      * Protobuf type {@code hbase.pb.Throttle}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Throttle)
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Throttle_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Throttle_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1709,12 +1709,12 @@ public final class QuotaProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getReqNumFieldBuilder();
           getReqSizeFieldBuilder();
@@ -1765,7 +1765,7 @@ public final class QuotaProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Throttle_descriptor;
       }
@@ -1843,29 +1843,29 @@ public final class QuotaProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle)other);
         } else {
@@ -1934,13 +1934,13 @@ public final class QuotaProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1953,7 +1953,7 @@ public final class QuotaProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota reqNum_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> reqNumBuilder_;
       /**
        * optional .hbase.pb.TimedQuota req_num = 1;
@@ -2056,11 +2056,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota req_num = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getReqNumFieldBuilder() {
         if (reqNumBuilder_ == null) {
-          reqNumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          reqNumBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getReqNum(),
                   getParentForChildren(),
@@ -2071,7 +2071,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota reqSize_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> reqSizeBuilder_;
       /**
        * optional .hbase.pb.TimedQuota req_size = 2;
@@ -2174,11 +2174,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota req_size = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getReqSizeFieldBuilder() {
         if (reqSizeBuilder_ == null) {
-          reqSizeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          reqSizeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getReqSize(),
                   getParentForChildren(),
@@ -2189,7 +2189,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota writeNum_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> writeNumBuilder_;
       /**
        * optional .hbase.pb.TimedQuota write_num = 3;
@@ -2292,11 +2292,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota write_num = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getWriteNumFieldBuilder() {
         if (writeNumBuilder_ == null) {
-          writeNumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          writeNumBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getWriteNum(),
                   getParentForChildren(),
@@ -2307,7 +2307,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota writeSize_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> writeSizeBuilder_;
       /**
        * optional .hbase.pb.TimedQuota write_size = 4;
@@ -2410,11 +2410,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota write_size = 4;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getWriteSizeFieldBuilder() {
         if (writeSizeBuilder_ == null) {
-          writeSizeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          writeSizeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getWriteSize(),
                   getParentForChildren(),
@@ -2425,7 +2425,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota readNum_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> readNumBuilder_;
       /**
        * optional .hbase.pb.TimedQuota read_num = 5;
@@ -2528,11 +2528,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota read_num = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getReadNumFieldBuilder() {
         if (readNumBuilder_ == null) {
-          readNumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          readNumBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getReadNum(),
                   getParentForChildren(),
@@ -2543,7 +2543,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota readSize_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> readSizeBuilder_;
       /**
        * optional .hbase.pb.TimedQuota read_size = 6;
@@ -2646,11 +2646,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota read_size = 6;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getReadSizeFieldBuilder() {
         if (readSizeBuilder_ == null) {
-          readSizeBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          readSizeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getReadSize(),
                   getParentForChildren(),
@@ -2660,12 +2660,12 @@ public final class QuotaProtos {
         return readSizeBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2683,22 +2683,22 @@ public final class QuotaProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Throttle parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Throttle(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2710,7 +2710,7 @@ public final class QuotaProtos {
 
   public interface ThrottleRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ThrottleRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.ThrottleType type = 1;
@@ -2738,11 +2738,11 @@ public final class QuotaProtos {
    * Protobuf type {@code hbase.pb.ThrottleRequest}
    */
   public  static final class ThrottleRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ThrottleRequest)
       ThrottleRequestOrBuilder {
     // Use ThrottleRequest.newBuilder() to construct.
-    private ThrottleRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ThrottleRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ThrottleRequest() {
@@ -2750,18 +2750,18 @@ public final class QuotaProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ThrottleRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2803,22 +2803,22 @@ public final class QuotaProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_ThrottleRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2879,7 +2879,7 @@ public final class QuotaProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, type_);
@@ -2896,11 +2896,11 @@ public final class QuotaProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, type_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTimedQuota());
       }
       size += unknownFields.getSerializedSize();
@@ -2954,61 +2954,61 @@ public final class QuotaProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3026,7 +3026,7 @@ public final class QuotaProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3034,15 +3034,15 @@ public final class QuotaProtos {
      * Protobuf type {@code hbase.pb.ThrottleRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ThrottleRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_ThrottleRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3055,12 +3055,12 @@ public final class QuotaProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTimedQuotaFieldBuilder();
         }
@@ -3078,7 +3078,7 @@ public final class QuotaProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_ThrottleRequest_descriptor;
       }
@@ -3120,29 +3120,29 @@ public final class QuotaProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest)other);
         } else {
@@ -3174,13 +3174,13 @@ public final class QuotaProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3229,7 +3229,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota timedQuota_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> timedQuotaBuilder_;
       /**
        * optional .hbase.pb.TimedQuota timed_quota = 2;
@@ -3332,11 +3332,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.TimedQuota timed_quota = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
           getTimedQuotaFieldBuilder() {
         if (timedQuotaBuilder_ == null) {
-          timedQuotaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          timedQuotaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>(
                   getTimedQuota(),
                   getParentForChildren(),
@@ -3346,12 +3346,12 @@ public final class QuotaProtos {
         return timedQuotaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3369,22 +3369,22 @@ public final class QuotaProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ThrottleRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ThrottleRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3396,7 +3396,7 @@ public final class QuotaProtos {
 
   public interface QuotasOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Quotas)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool bypass_globals = 1 [default = false];
@@ -3424,11 +3424,11 @@ public final class QuotaProtos {
    * Protobuf type {@code hbase.pb.Quotas}
    */
   public  static final class Quotas extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Quotas)
       QuotasOrBuilder {
     // Use Quotas.newBuilder() to construct.
-    private Quotas(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Quotas(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Quotas() {
@@ -3436,18 +3436,18 @@ public final class QuotaProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Quotas(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3483,22 +3483,22 @@ public final class QuotaProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Quotas_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Quotas_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3558,7 +3558,7 @@ public final class QuotaProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, bypassGlobals_);
@@ -3575,11 +3575,11 @@ public final class QuotaProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, bypassGlobals_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getThrottle());
       }
       size += unknownFields.getSerializedSize();
@@ -3622,7 +3622,7 @@ public final class QuotaProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasBypassGlobals()) {
         hash = (37 * hash) + BYPASS_GLOBALS_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getBypassGlobals());
       }
       if (hasThrottle()) {
@@ -3635,61 +3635,61 @@ public final class QuotaProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3707,7 +3707,7 @@ public final class QuotaProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3715,15 +3715,15 @@ public final class QuotaProtos {
      * Protobuf type {@code hbase.pb.Quotas}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Quotas)
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotasOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Quotas_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Quotas_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3736,12 +3736,12 @@ public final class QuotaProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getThrottleFieldBuilder();
         }
@@ -3759,7 +3759,7 @@ public final class QuotaProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_Quotas_descriptor;
       }
@@ -3801,29 +3801,29 @@ public final class QuotaProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas)other);
         } else {
@@ -3855,13 +3855,13 @@ public final class QuotaProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3906,7 +3906,7 @@ public final class QuotaProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle throttle_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder> throttleBuilder_;
       /**
        * optional .hbase.pb.Throttle throttle = 2;
@@ -4009,11 +4009,11 @@ public final class QuotaProtos {
       /**
        * optional .hbase.pb.Throttle throttle = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder> 
           getThrottleFieldBuilder() {
         if (throttleBuilder_ == null) {
-          throttleBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          throttleBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder>(
                   getThrottle(),
                   getParentForChildren(),
@@ -4023,12 +4023,12 @@ public final class QuotaProtos {
         return throttleBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4046,22 +4046,22 @@ public final class QuotaProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Quotas parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Quotas(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4073,34 +4073,34 @@ public final class QuotaProtos {
 
   public interface QuotaUsageOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.QuotaUsage)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.QuotaUsage}
    */
   public  static final class QuotaUsage extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.QuotaUsage)
       QuotaUsageOrBuilder {
     // Use QuotaUsage.newBuilder() to construct.
-    private QuotaUsage(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private QuotaUsage(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private QuotaUsage() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private QuotaUsage(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4118,22 +4118,22 @@ public final class QuotaProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_QuotaUsage_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_QuotaUsage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4150,7 +4150,7 @@ public final class QuotaProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -4194,61 +4194,61 @@ public final class QuotaProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4266,7 +4266,7 @@ public final class QuotaProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4274,15 +4274,15 @@ public final class QuotaProtos {
      * Protobuf type {@code hbase.pb.QuotaUsage}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.QuotaUsage)
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsageOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_QuotaUsage_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_QuotaUsage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4295,12 +4295,12 @@ public final class QuotaProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -4309,7 +4309,7 @@ public final class QuotaProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.internal_static_hbase_pb_QuotaUsage_descriptor;
       }
@@ -4336,29 +4336,29 @@ public final class QuotaProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage)other);
         } else {
@@ -4379,13 +4379,13 @@ public final class QuotaProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.QuotaUsage) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4396,12 +4396,12 @@ public final class QuotaProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4419,22 +4419,22 @@ public final class QuotaProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public QuotaUsage parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new QuotaUsage(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4444,37 +4444,37 @@ public final class QuotaProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimedQuota_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Throttle_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Throttle_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ThrottleRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Quotas_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Quotas_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_QuotaUsage_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -4502,47 +4502,47 @@ public final class QuotaProtos {
       "d.protobuf.generatedB\013QuotaProtosH\001\210\001\001\240\001" +
       "\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_TimedQuota_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_TimedQuota_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TimedQuota_descriptor,
         new java.lang.String[] { "TimeUnit", "SoftLimit", "Share", "Scope", });
     internal_static_hbase_pb_Throttle_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_Throttle_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Throttle_descriptor,
         new java.lang.String[] { "ReqNum", "ReqSize", "WriteNum", "WriteSize", "ReadNum", "ReadSize", });
     internal_static_hbase_pb_ThrottleRequest_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ThrottleRequest_descriptor,
         new java.lang.String[] { "Type", "TimedQuota", });
     internal_static_hbase_pb_Quotas_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_Quotas_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Quotas_descriptor,
         new java.lang.String[] { "BypassGlobals", "Throttle", });
     internal_static_hbase_pb_QuotaUsage_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_QuotaUsage_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_QuotaUsage_descriptor,
         new java.lang.String[] { });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
index ead61ed..3c2bec0 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class RPCProtos {
   private RPCProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface UserInformationOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.UserInformation)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required string effective_user = 1;
@@ -29,7 +29,7 @@ public final class RPCProtos {
     /**
      * required string effective_user = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getEffectiveUserBytes();
 
     /**
@@ -43,7 +43,7 @@ public final class RPCProtos {
     /**
      * optional string real_user = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRealUserBytes();
   }
   /**
@@ -54,11 +54,11 @@ public final class RPCProtos {
    * Protobuf type {@code hbase.pb.UserInformation}
    */
   public  static final class UserInformation extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.UserInformation)
       UserInformationOrBuilder {
     // Use UserInformation.newBuilder() to construct.
-    private UserInformation(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private UserInformation(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private UserInformation() {
@@ -67,18 +67,18 @@ public final class RPCProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private UserInformation(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -95,35 +95,35 @@ public final class RPCProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               effectiveUser_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               realUser_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_UserInformation_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_UserInformation_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -147,8 +147,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           effectiveUser_ = s;
@@ -159,17 +159,17 @@ public final class RPCProtos {
     /**
      * required string effective_user = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getEffectiveUserBytes() {
       java.lang.Object ref = effectiveUser_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         effectiveUser_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -189,8 +189,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           realUser_ = s;
@@ -201,17 +201,17 @@ public final class RPCProtos {
     /**
      * optional string real_user = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRealUserBytes() {
       java.lang.Object ref = realUser_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         realUser_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -229,13 +229,13 @@ public final class RPCProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, effectiveUser_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, effectiveUser_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, realUser_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, realUser_);
       }
       unknownFields.writeTo(output);
     }
@@ -246,10 +246,10 @@ public final class RPCProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, effectiveUser_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, effectiveUser_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, realUser_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, realUser_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -303,61 +303,61 @@ public final class RPCProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -375,7 +375,7 @@ public final class RPCProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -387,15 +387,15 @@ public final class RPCProtos {
      * Protobuf type {@code hbase.pb.UserInformation}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.UserInformation)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_UserInformation_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_UserInformation_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -408,12 +408,12 @@ public final class RPCProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -426,7 +426,7 @@ public final class RPCProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_UserInformation_descriptor;
       }
@@ -464,29 +464,29 @@ public final class RPCProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation)other);
         } else {
@@ -520,13 +520,13 @@ public final class RPCProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -551,8 +551,8 @@ public final class RPCProtos {
       public java.lang.String getEffectiveUser() {
         java.lang.Object ref = effectiveUser_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             effectiveUser_ = s;
@@ -565,17 +565,17 @@ public final class RPCProtos {
       /**
        * required string effective_user = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getEffectiveUserBytes() {
         java.lang.Object ref = effectiveUser_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           effectiveUser_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -604,7 +604,7 @@ public final class RPCProtos {
        * required string effective_user = 1;
        */
       public Builder setEffectiveUserBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -627,8 +627,8 @@ public final class RPCProtos {
       public java.lang.String getRealUser() {
         java.lang.Object ref = realUser_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             realUser_ = s;
@@ -641,17 +641,17 @@ public final class RPCProtos {
       /**
        * optional string real_user = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getRealUserBytes() {
         java.lang.Object ref = realUser_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           realUser_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -680,7 +680,7 @@ public final class RPCProtos {
        * optional string real_user = 2;
        */
       public Builder setRealUserBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -690,12 +690,12 @@ public final class RPCProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -713,22 +713,22 @@ public final class RPCProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public UserInformation parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new UserInformation(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -740,7 +740,7 @@ public final class RPCProtos {
 
   public interface ConnectionHeaderOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ConnectionHeader)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.UserInformation user_info = 1;
@@ -766,7 +766,7 @@ public final class RPCProtos {
     /**
      * optional string service_name = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getServiceNameBytes();
 
     /**
@@ -795,7 +795,7 @@ public final class RPCProtos {
      *
      * optional string cell_block_codec_class = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCellBlockCodecClassBytes();
 
     /**
@@ -824,7 +824,7 @@ public final class RPCProtos {
      *
      * optional string cell_block_compressor_class = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCellBlockCompressorClassBytes();
 
     /**
@@ -848,11 +848,11 @@ public final class RPCProtos {
    * Protobuf type {@code hbase.pb.ConnectionHeader}
    */
   public  static final class ConnectionHeader extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ConnectionHeader)
       ConnectionHeaderOrBuilder {
     // Use ConnectionHeader.newBuilder() to construct.
-    private ConnectionHeader(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ConnectionHeader(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ConnectionHeader() {
@@ -862,18 +862,18 @@ public final class RPCProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ConnectionHeader(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -903,19 +903,19 @@ public final class RPCProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               serviceName_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               cellBlockCodecClass_ = bs;
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               cellBlockCompressorClass_ = bs;
               break;
@@ -935,22 +935,22 @@ public final class RPCProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeader_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeader_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -995,8 +995,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           serviceName_ = s;
@@ -1007,17 +1007,17 @@ public final class RPCProtos {
     /**
      * optional string service_name = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getServiceNameBytes() {
       java.lang.Object ref = serviceName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         serviceName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -1047,8 +1047,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           cellBlockCodecClass_ = s;
@@ -1064,17 +1064,17 @@ public final class RPCProtos {
      *
      * optional string cell_block_codec_class = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCellBlockCodecClassBytes() {
       java.lang.Object ref = cellBlockCodecClass_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         cellBlockCodecClass_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -1104,8 +1104,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           cellBlockCompressorClass_ = s;
@@ -1121,17 +1121,17 @@ public final class RPCProtos {
      *
      * optional string cell_block_compressor_class = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCellBlockCompressorClassBytes() {
       java.lang.Object ref = cellBlockCompressorClass_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         cellBlockCompressorClass_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -1178,19 +1178,19 @@ public final class RPCProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, serviceName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, serviceName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, cellBlockCodecClass_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, cellBlockCodecClass_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, cellBlockCompressorClass_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, cellBlockCompressorClass_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeMessage(5, getVersionInfo());
@@ -1204,20 +1204,20 @@ public final class RPCProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, serviceName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, serviceName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, cellBlockCodecClass_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, cellBlockCodecClass_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, cellBlockCompressorClass_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, cellBlockCompressorClass_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getVersionInfo());
       }
       size += unknownFields.getSerializedSize();
@@ -1299,61 +1299,61 @@ public final class RPCProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1371,7 +1371,7 @@ public final class RPCProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1383,15 +1383,15 @@ public final class RPCProtos {
      * Protobuf type {@code hbase.pb.ConnectionHeader}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ConnectionHeader)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeaderOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeader_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeader_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1404,12 +1404,12 @@ public final class RPCProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
           getVersionInfoFieldBuilder();
@@ -1438,7 +1438,7 @@ public final class RPCProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ConnectionHeader_descriptor;
       }
@@ -1496,29 +1496,29 @@ public final class RPCProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader)other);
         } else {
@@ -1570,13 +1570,13 @@ public final class RPCProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1589,7 +1589,7 @@ public final class RPCProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation userInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
       /**
        * optional .hbase.pb.UserInformation user_info = 1;
@@ -1692,11 +1692,11 @@ public final class RPCProtos {
       /**
        * optional .hbase.pb.UserInformation user_info = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
           getUserInfoFieldBuilder() {
         if (userInfoBuilder_ == null) {
-          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          userInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
                   getUserInfo(),
                   getParentForChildren(),
@@ -1719,8 +1719,8 @@ public final class RPCProtos {
       public java.lang.String getServiceName() {
         java.lang.Object ref = serviceName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             serviceName_ = s;
@@ -1733,17 +1733,17 @@ public final class RPCProtos {
       /**
        * optional string service_name = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getServiceNameBytes() {
         java.lang.Object ref = serviceName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           serviceName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1772,7 +1772,7 @@ public final class RPCProtos {
        * optional string service_name = 2;
        */
       public Builder setServiceNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1805,8 +1805,8 @@ public final class RPCProtos {
       public java.lang.String getCellBlockCodecClass() {
         java.lang.Object ref = cellBlockCodecClass_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             cellBlockCodecClass_ = s;
@@ -1824,17 +1824,17 @@ public final class RPCProtos {
        *
        * optional string cell_block_codec_class = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getCellBlockCodecClassBytes() {
         java.lang.Object ref = cellBlockCodecClass_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           cellBlockCodecClass_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1878,7 +1878,7 @@ public final class RPCProtos {
        * optional string cell_block_codec_class = 3;
        */
       public Builder setCellBlockCodecClassBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1911,8 +1911,8 @@ public final class RPCProtos {
       public java.lang.String getCellBlockCompressorClass() {
         java.lang.Object ref = cellBlockCompressorClass_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             cellBlockCompressorClass_ = s;
@@ -1930,17 +1930,17 @@ public final class RPCProtos {
        *
        * optional string cell_block_compressor_class = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getCellBlockCompressorClassBytes() {
         java.lang.Object ref = cellBlockCompressorClass_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           cellBlockCompressorClass_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1984,7 +1984,7 @@ public final class RPCProtos {
        * optional string cell_block_compressor_class = 4;
        */
       public Builder setCellBlockCompressorClassBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1995,7 +1995,7 @@ public final class RPCProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo versionInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder> versionInfoBuilder_;
       /**
        * optional .hbase.pb.VersionInfo version_info = 5;
@@ -2098,11 +2098,11 @@ public final class RPCProtos {
       /**
        * optional .hbase.pb.VersionInfo version_info = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder> 
           getVersionInfoFieldBuilder() {
         if (versionInfoBuilder_ == null) {
-          versionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          versionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfoOrBuilder>(
                   getVersionInfo(),
                   getParentForChildren(),
@@ -2112,12 +2112,12 @@ public final class RPCProtos {
         return versionInfoBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2135,22 +2135,22 @@ public final class RPCProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ConnectionHeader parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ConnectionHeader(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2162,7 +2162,7 @@ public final class RPCProtos {
 
   public interface CellBlockMetaOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CellBlockMeta)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -2189,11 +2189,11 @@ public final class RPCProtos {
    * Protobuf type {@code hbase.pb.CellBlockMeta}
    */
   public  static final class CellBlockMeta extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CellBlockMeta)
       CellBlockMetaOrBuilder {
     // Use CellBlockMeta.newBuilder() to construct.
-    private CellBlockMeta(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CellBlockMeta(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CellBlockMeta() {
@@ -2201,18 +2201,18 @@ public final class RPCProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CellBlockMeta(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2235,22 +2235,22 @@ public final class RPCProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CellBlockMeta_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CellBlockMeta_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2291,7 +2291,7 @@ public final class RPCProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, length_);
@@ -2305,7 +2305,7 @@ public final class RPCProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, length_);
       }
       size += unknownFields.getSerializedSize();
@@ -2351,61 +2351,61 @@ public final class RPCProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2423,7 +2423,7 @@ public final class RPCProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2435,15 +2435,15 @@ public final class RPCProtos {
      * Protobuf type {@code hbase.pb.CellBlockMeta}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CellBlockMeta)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CellBlockMeta_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CellBlockMeta_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2456,12 +2456,12 @@ public final class RPCProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -2472,7 +2472,7 @@ public final class RPCProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_CellBlockMeta_descriptor;
       }
@@ -2506,29 +2506,29 @@ public final class RPCProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta)other);
         } else {
@@ -2552,13 +2552,13 @@ public final class RPCProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2618,12 +2618,12 @@ public final class RPCProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2641,22 +2641,22 @@ public final class RPCProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CellBlockMeta parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CellBlockMeta(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2668,7 +2668,7 @@ public final class RPCProtos {
 
   public interface ExceptionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ExceptionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -2693,7 +2693,7 @@ public final class RPCProtos {
      *
      * optional string exception_class_name = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getExceptionClassNameBytes();
 
     /**
@@ -2719,7 +2719,7 @@ public final class RPCProtos {
      *
      * optional string stack_trace = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStackTraceBytes();
 
     /**
@@ -2748,7 +2748,7 @@ public final class RPCProtos {
      *
      * optional string hostname = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getHostnameBytes();
 
     /**
@@ -2786,11 +2786,11 @@ public final class RPCProtos {
    * Protobuf type {@code hbase.pb.ExceptionResponse}
    */
   public  static final class ExceptionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ExceptionResponse)
       ExceptionResponseOrBuilder {
     // Use ExceptionResponse.newBuilder() to construct.
-    private ExceptionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ExceptionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ExceptionResponse() {
@@ -2802,18 +2802,18 @@ public final class RPCProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ExceptionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2830,19 +2830,19 @@ public final class RPCProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               exceptionClassName_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               stackTrace_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               hostname_ = bs;
               break;
@@ -2859,22 +2859,22 @@ public final class RPCProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ExceptionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ExceptionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2906,8 +2906,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           exceptionClassName_ = s;
@@ -2922,17 +2922,17 @@ public final class RPCProtos {
      *
      * optional string exception_class_name = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getExceptionClassNameBytes() {
       java.lang.Object ref = exceptionClassName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         exceptionClassName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -2960,8 +2960,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           stackTrace_ = s;
@@ -2976,17 +2976,17 @@ public final class RPCProtos {
      *
      * optional string stack_trace = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStackTraceBytes() {
       java.lang.Object ref = stackTrace_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         stackTrace_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -3016,8 +3016,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           hostname_ = s;
@@ -3033,17 +3033,17 @@ public final class RPCProtos {
      *
      * optional string hostname = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getHostnameBytes() {
       java.lang.Object ref = hostname_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         hostname_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -3095,16 +3095,16 @@ public final class RPCProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, exceptionClassName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, exceptionClassName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, stackTrace_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, stackTrace_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, hostname_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, hostname_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeInt32(4, port_);
@@ -3121,20 +3121,20 @@ public final class RPCProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, exceptionClassName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, exceptionClassName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, stackTrace_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, stackTrace_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, hostname_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, hostname_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(4, port_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(5, doNotRetry_);
       }
       size += unknownFields.getSerializedSize();
@@ -3208,7 +3208,7 @@ public final class RPCProtos {
       }
       if (hasDoNotRetry()) {
         hash = (37 * hash) + DO_NOT_RETRY_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDoNotRetry());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -3217,61 +3217,61 @@ public final class RPCProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3289,7 +3289,7 @@ public final class RPCProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3302,15 +3302,15 @@ public final class RPCProtos {
      * Protobuf type {@code hbase.pb.ExceptionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ExceptionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ExceptionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ExceptionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3323,12 +3323,12 @@ public final class RPCProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3347,7 +3347,7 @@ public final class RPCProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ExceptionResponse_descriptor;
       }
@@ -3397,29 +3397,29 @@ public final class RPCProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse)other);
         } else {
@@ -3461,13 +3461,13 @@ public final class RPCProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3500,8 +3500,8 @@ public final class RPCProtos {
       public java.lang.String getExceptionClassName() {
         java.lang.Object ref = exceptionClassName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             exceptionClassName_ = s;
@@ -3518,17 +3518,17 @@ public final class RPCProtos {
        *
        * optional string exception_class_name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getExceptionClassNameBytes() {
         java.lang.Object ref = exceptionClassName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           exceptionClassName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -3569,7 +3569,7 @@ public final class RPCProtos {
        * optional string exception_class_name = 1;
        */
       public Builder setExceptionClassNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3600,8 +3600,8 @@ public final class RPCProtos {
       public java.lang.String getStackTrace() {
         java.lang.Object ref = stackTrace_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             stackTrace_ = s;
@@ -3618,17 +3618,17 @@ public final class RPCProtos {
        *
        * optional string stack_trace = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStackTraceBytes() {
         java.lang.Object ref = stackTrace_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           stackTrace_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -3669,7 +3669,7 @@ public final class RPCProtos {
        * optional string stack_trace = 2;
        */
       public Builder setStackTraceBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3702,8 +3702,8 @@ public final class RPCProtos {
       public java.lang.String getHostname() {
         java.lang.Object ref = hostname_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             hostname_ = s;
@@ -3721,17 +3721,17 @@ public final class RPCProtos {
        *
        * optional string hostname = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getHostnameBytes() {
         java.lang.Object ref = hostname_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           hostname_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -3775,7 +3775,7 @@ public final class RPCProtos {
        * optional string hostname = 3;
        */
       public Builder setHostnameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3865,12 +3865,12 @@ public final class RPCProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3888,22 +3888,22 @@ public final class RPCProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ExceptionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ExceptionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3915,7 +3915,7 @@ public final class RPCProtos {
 
   public interface RequestHeaderOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RequestHeader)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -3958,7 +3958,7 @@ public final class RPCProtos {
     /**
      * optional string method_name = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMethodNameBytes();
 
     /**
@@ -4039,11 +4039,11 @@ public final class RPCProtos {
    * Protobuf type {@code hbase.pb.RequestHeader}
    */
   public  static final class RequestHeader extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RequestHeader)
       RequestHeaderOrBuilder {
     // Use RequestHeader.newBuilder() to construct.
-    private RequestHeader(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RequestHeader(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RequestHeader() {
@@ -4055,18 +4055,18 @@ public final class RPCProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RequestHeader(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4101,7 +4101,7 @@ public final class RPCProtos {
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               methodName_ = bs;
               break;
@@ -4136,22 +4136,22 @@ public final class RPCProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_RequestHeader_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_RequestHeader_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4219,8 +4219,8 @@ public final class RPCProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           methodName_ = s;
@@ -4231,17 +4231,17 @@ public final class RPCProtos {
     /**
      * optional string method_name = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getMethodNameBytes() {
       java.lang.Object ref = methodName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         methodName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -4351,7 +4351,7 @@ public final class RPCProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, callId_);
@@ -4360,7 +4360,7 @@ public final class RPCProtos {
         output.writeMessage(2, getTraceInfo());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, methodName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, methodName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeBool(4, requestParam_);
@@ -4383,30 +4383,30 @@ public final class RPCProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, callId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getTraceInfo());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, methodName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, methodName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, requestParam_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getCellBlockMeta());
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(6, priority_);
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(7, timeout_);
       }
       size += unknownFields.getSerializedSize();
@@ -4486,7 +4486,7 @@ public final class RPCProtos {
       }
       if (hasRequestParam()) {
         hash = (37 * hash) + REQUEST_PARAM_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getRequestParam());
       }
       if (hasCellBlockMeta()) {
@@ -4507,61 +4507,61 @@ public final class RPCProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4579,7 +4579,7 @@ public final class RPCProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4591,15 +4591,15 @@ public final class RPCProtos {
      * Protobuf type {@code hbase.pb.RequestHeader}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RequestHeader)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeaderOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_RequestHeader_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_RequestHeader_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4612,12 +4612,12 @@ public final class RPCProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTraceInfoFieldBuilder();
           getCellBlockMetaFieldBuilder();
@@ -4650,7 +4650,7 @@ public final class RPCProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_RequestHeader_descriptor;
       }
@@ -4716,29 +4716,29 @@ public final class RPCProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader)other);
         } else {
@@ -4782,13 +4782,13 @@ public final class RPCProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4849,7 +4849,7 @@ public final class RPCProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo traceInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfoOrBuilder> traceInfoBuilder_;
       /**
        * optional .hbase.pb.RPCTInfo trace_info = 2;
@@ -4952,11 +4952,11 @@ public final class RPCProtos {
       /**
        * optional .hbase.pb.RPCTInfo trace_info = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfoOrBuilder> 
           getTraceInfoFieldBuilder() {
         if (traceInfoBuilder_ == null) {
-          traceInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          traceInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfoOrBuilder>(
                   getTraceInfo(),
                   getParentForChildren(),
@@ -4979,8 +4979,8 @@ public final class RPCProtos {
       public java.lang.String getMethodName() {
         java.lang.Object ref = methodName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             methodName_ = s;
@@ -4993,17 +4993,17 @@ public final class RPCProtos {
       /**
        * optional string method_name = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getMethodNameBytes() {
         java.lang.Object ref = methodName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           methodName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -5032,7 +5032,7 @@ public final class RPCProtos {
        * optional string method_name = 3;
        */
       public Builder setMethodNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5091,7 +5091,7 @@ public final class RPCProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta cellBlockMeta_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> cellBlockMetaBuilder_;
       /**
        * 
@@ -5230,11 +5230,11 @@ public final class RPCProtos {
        *
        * optional .hbase.pb.CellBlockMeta cell_block_meta = 5;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> 
           getCellBlockMetaFieldBuilder() {
         if (cellBlockMetaBuilder_ == null) {
-          cellBlockMetaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          cellBlockMetaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder>(
                   getCellBlockMeta(),
                   getParentForChildren(),
@@ -5328,12 +5328,12 @@ public final class RPCProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5351,22 +5351,22 @@ public final class RPCProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RequestHeader parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RequestHeader(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5378,7 +5378,7 @@ public final class RPCProtos {
 
   public interface ResponseHeaderOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ResponseHeader)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional uint32 call_id = 1;
@@ -5443,11 +5443,11 @@ public final class RPCProtos {
    * Protobuf type {@code hbase.pb.ResponseHeader}
    */
   public  static final class ResponseHeader extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ResponseHeader)
       ResponseHeaderOrBuilder {
     // Use ResponseHeader.newBuilder() to construct.
-    private ResponseHeader(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ResponseHeader(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ResponseHeader() {
@@ -5455,18 +5455,18 @@ public final class RPCProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ResponseHeader(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5515,22 +5515,22 @@ public final class RPCProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ResponseHeader_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ResponseHeader_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5629,7 +5629,7 @@ public final class RPCProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, callId_);
@@ -5649,15 +5649,15 @@ public final class RPCProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, callId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getException());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, getCellBlockMeta());
       }
       size += unknownFields.getSerializedSize();
@@ -5721,61 +5721,61 @@ public final class RPCProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5793,7 +5793,7 @@ public final class RPCProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5801,15 +5801,15 @@ public final class RPCProtos {
      * Protobuf type {@code hbase.pb.ResponseHeader}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ResponseHeader)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeaderOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ResponseHeader_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ResponseHeader_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5822,12 +5822,12 @@ public final class RPCProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getExceptionFieldBuilder();
           getCellBlockMetaFieldBuilder();
@@ -5852,7 +5852,7 @@ public final class RPCProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.internal_static_hbase_pb_ResponseHeader_descriptor;
       }
@@ -5902,29 +5902,29 @@ public final class RPCProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader)other);
         } else {
@@ -5954,13 +5954,13 @@ public final class RPCProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6005,7 +6005,7 @@ public final class RPCProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse exception_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder> exceptionBuilder_;
       /**
        * 
@@ -6144,11 +6144,11 @@ public final class RPCProtos {
        *
        * optional .hbase.pb.ExceptionResponse exception = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder> 
           getExceptionFieldBuilder() {
         if (exceptionBuilder_ == null) {
-          exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          exceptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponseOrBuilder>(
                   getException(),
                   getParentForChildren(),
@@ -6159,7 +6159,7 @@ public final class RPCProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta cellBlockMeta_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> cellBlockMetaBuilder_;
       /**
        * 
@@ -6298,11 +6298,11 @@ public final class RPCProtos {
        *
        * optional .hbase.pb.CellBlockMeta cell_block_meta = 3;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder> 
           getCellBlockMetaFieldBuilder() {
         if (cellBlockMetaBuilder_ == null) {
-          cellBlockMetaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          cellBlockMetaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder>(
                   getCellBlockMeta(),
                   getParentForChildren(),
@@ -6312,12 +6312,12 @@ public final class RPCProtos {
         return cellBlockMetaBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6335,22 +6335,22 @@ public final class RPCProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ResponseHeader parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ResponseHeader(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6360,42 +6360,42 @@ public final class RPCProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UserInformation_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UserInformation_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ConnectionHeader_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ConnectionHeader_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CellBlockMeta_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CellBlockMeta_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ExceptionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ExceptionResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RequestHeader_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RequestHeader_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ResponseHeader_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ResponseHeader_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -6422,54 +6422,54 @@ public final class RPCProtos {
       "llBlockMetaBC\n1org.apache.hadoop.hbase.s" +
       "haded.protobuf.generatedB\tRPCProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_UserInformation_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_UserInformation_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_UserInformation_descriptor,
         new java.lang.String[] { "EffectiveUser", "RealUser", });
     internal_static_hbase_pb_ConnectionHeader_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_ConnectionHeader_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ConnectionHeader_descriptor,
         new java.lang.String[] { "UserInfo", "ServiceName", "CellBlockCodecClass", "CellBlockCompressorClass", "VersionInfo", });
     internal_static_hbase_pb_CellBlockMeta_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_CellBlockMeta_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CellBlockMeta_descriptor,
         new java.lang.String[] { "Length", });
     internal_static_hbase_pb_ExceptionResponse_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_ExceptionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ExceptionResponse_descriptor,
         new java.lang.String[] { "ExceptionClassName", "StackTrace", "Hostname", "Port", "DoNotRetry", });
     internal_static_hbase_pb_RequestHeader_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_RequestHeader_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RequestHeader_descriptor,
         new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", "Priority", "Timeout", });
     internal_static_hbase_pb_ResponseHeader_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_ResponseHeader_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ResponseHeader_descriptor,
         new java.lang.String[] { "CallId", "Exception", "CellBlockMeta", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
index 66b6990..0a58113 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class RegionNormalizerProtos {
   private RegionNormalizerProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface RegionNormalizerStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:RegionNormalizerState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool normalizer_on = 1;
@@ -31,11 +31,11 @@ public final class RegionNormalizerProtos {
    * Protobuf type {@code RegionNormalizerState}
    */
   public  static final class RegionNormalizerState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:RegionNormalizerState)
       RegionNormalizerStateOrBuilder {
     // Use RegionNormalizerState.newBuilder() to construct.
-    private RegionNormalizerState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionNormalizerState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionNormalizerState() {
@@ -43,18 +43,18 @@ public final class RegionNormalizerProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionNormalizerState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -77,22 +77,22 @@ public final class RegionNormalizerProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.internal_static_RegionNormalizerState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.internal_static_RegionNormalizerState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -125,7 +125,7 @@ public final class RegionNormalizerProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, normalizerOn_);
@@ -139,7 +139,7 @@ public final class RegionNormalizerProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, normalizerOn_);
       }
       size += unknownFields.getSerializedSize();
@@ -177,7 +177,7 @@ public final class RegionNormalizerProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasNormalizerOn()) {
         hash = (37 * hash) + NORMALIZER_ON_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getNormalizerOn());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -186,61 +186,61 @@ public final class RegionNormalizerProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -258,7 +258,7 @@ public final class RegionNormalizerProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -266,15 +266,15 @@ public final class RegionNormalizerProtos {
      * Protobuf type {@code RegionNormalizerState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:RegionNormalizerState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.internal_static_RegionNormalizerState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.internal_static_RegionNormalizerState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -287,12 +287,12 @@ public final class RegionNormalizerProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -303,7 +303,7 @@ public final class RegionNormalizerProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.internal_static_RegionNormalizerState_descriptor;
       }
@@ -337,29 +337,29 @@ public final class RegionNormalizerProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState)other);
         } else {
@@ -383,13 +383,13 @@ public final class RegionNormalizerProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos.RegionNormalizerState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -433,12 +433,12 @@ public final class RegionNormalizerProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -456,22 +456,22 @@ public final class RegionNormalizerProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionNormalizerState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionNormalizerState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -481,17 +481,17 @@ public final class RegionNormalizerProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_RegionNormalizerState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_RegionNormalizerState_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -500,22 +500,22 @@ public final class RegionNormalizerProtos {
       "apache.hadoop.hbase.shaded.protobuf.gene" +
       "ratedB\026RegionNormalizerProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_RegionNormalizerState_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_RegionNormalizerState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_RegionNormalizerState_descriptor,
         new java.lang.String[] { "NormalizerOn", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
index 540df5a..b8712b7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class RegionServerStatusProtos {
   private RegionServerStatusProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface RegionServerStartupRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionServerStartupRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -92,18 +92,18 @@ public final class RegionServerStatusProtos {
      *
      * optional string use_this_hostname_instead = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUseThisHostnameInsteadBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.RegionServerStartupRequest}
    */
   public  static final class RegionServerStartupRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionServerStartupRequest)
       RegionServerStartupRequestOrBuilder {
     // Use RegionServerStartupRequest.newBuilder() to construct.
-    private RegionServerStartupRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionServerStartupRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionServerStartupRequest() {
@@ -114,18 +114,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionServerStartupRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -157,29 +157,29 @@ public final class RegionServerStatusProtos {
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               useThisHostnameInstead_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -280,8 +280,8 @@ public final class RegionServerStatusProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           useThisHostnameInstead_ = s;
@@ -296,17 +296,17 @@ public final class RegionServerStatusProtos {
      *
      * optional string use_this_hostname_instead = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getUseThisHostnameInsteadBytes() {
       java.lang.Object ref = useThisHostnameInstead_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         useThisHostnameInstead_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -332,7 +332,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt32(1, port_);
@@ -344,7 +344,7 @@ public final class RegionServerStatusProtos {
         output.writeUInt64(3, serverCurrentTime_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, useThisHostnameInstead_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, useThisHostnameInstead_);
       }
       unknownFields.writeTo(output);
     }
@@ -355,19 +355,19 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(1, port_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(2, serverStartCode_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, serverCurrentTime_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, useThisHostnameInstead_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, useThisHostnameInstead_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -423,12 +423,12 @@ public final class RegionServerStatusProtos {
       }
       if (hasServerStartCode()) {
         hash = (37 * hash) + SERVER_START_CODE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getServerStartCode());
       }
       if (hasServerCurrentTime()) {
         hash = (37 * hash) + SERVER_CURRENT_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getServerCurrentTime());
       }
       if (hasUseThisHostnameInstead()) {
@@ -441,61 +441,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -513,7 +513,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -521,15 +521,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.RegionServerStartupRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionServerStartupRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -542,12 +542,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -564,7 +564,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupRequest_descriptor;
       }
@@ -610,29 +610,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest)other);
         } else {
@@ -676,13 +676,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -859,8 +859,8 @@ public final class RegionServerStatusProtos {
       public java.lang.String getUseThisHostnameInstead() {
         java.lang.Object ref = useThisHostnameInstead_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             useThisHostnameInstead_ = s;
@@ -877,17 +877,17 @@ public final class RegionServerStatusProtos {
        *
        * optional string use_this_hostname_instead = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getUseThisHostnameInsteadBytes() {
         java.lang.Object ref = useThisHostnameInstead_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           useThisHostnameInstead_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -928,7 +928,7 @@ public final class RegionServerStatusProtos {
        * optional string use_this_hostname_instead = 4;
        */
       public Builder setUseThisHostnameInsteadBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -938,12 +938,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -961,22 +961,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionServerStartupRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionServerStartupRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -988,7 +988,7 @@ public final class RegionServerStatusProtos {
 
   public interface RegionServerStartupResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionServerStartupResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -1053,11 +1053,11 @@ public final class RegionServerStatusProtos {
    * Protobuf type {@code hbase.pb.RegionServerStartupResponse}
    */
   public  static final class RegionServerStartupResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionServerStartupResponse)
       RegionServerStartupResponseOrBuilder {
     // Use RegionServerStartupResponse.newBuilder() to construct.
-    private RegionServerStartupResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionServerStartupResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionServerStartupResponse() {
@@ -1065,18 +1065,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionServerStartupResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1103,10 +1103,10 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -1116,12 +1116,12 @@ public final class RegionServerStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1214,7 +1214,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       for (int i = 0; i < mapEntries_.size(); i++) {
         output.writeMessage(1, mapEntries_.get(i));
@@ -1228,7 +1228,7 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       for (int i = 0; i < mapEntries_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, mapEntries_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -1271,61 +1271,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1343,7 +1343,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1351,15 +1351,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.RegionServerStartupResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionServerStartupResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1372,12 +1372,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getMapEntriesFieldBuilder();
         }
@@ -1393,7 +1393,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerStartupResponse_descriptor;
       }
@@ -1430,29 +1430,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse)other);
         } else {
@@ -1482,7 +1482,7 @@ public final class RegionServerStatusProtos {
               mapEntries_ = other.mapEntries_;
               bitField0_ = (bitField0_ & ~0x00000001);
               mapEntriesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getMapEntriesFieldBuilder() : null;
             } else {
               mapEntriesBuilder_.addAllMessages(other.mapEntries_);
@@ -1504,13 +1504,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1531,7 +1531,7 @@ public final class RegionServerStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> mapEntriesBuilder_;
 
       /**
@@ -1733,7 +1733,7 @@ public final class RegionServerStatusProtos {
           java.lang.Iterable values) {
         if (mapEntriesBuilder_ == null) {
           ensureMapEntriesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, mapEntries_);
           onChanged();
         } else {
@@ -1873,11 +1873,11 @@ public final class RegionServerStatusProtos {
            getMapEntriesBuilderList() {
         return getMapEntriesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
           getMapEntriesFieldBuilder() {
         if (mapEntriesBuilder_ == null) {
-          mapEntriesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          mapEntriesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder>(
                   mapEntries_,
                   ((bitField0_ & 0x00000001) == 0x00000001),
@@ -1888,12 +1888,12 @@ public final class RegionServerStatusProtos {
         return mapEntriesBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1911,22 +1911,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionServerStartupResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionServerStartupResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1938,7 +1938,7 @@ public final class RegionServerStatusProtos {
 
   public interface RegionServerReportRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionServerReportRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ServerName server = 1;
@@ -1982,29 +1982,29 @@ public final class RegionServerStatusProtos {
    * Protobuf type {@code hbase.pb.RegionServerReportRequest}
    */
   public  static final class RegionServerReportRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionServerReportRequest)
       RegionServerReportRequestOrBuilder {
     // Use RegionServerReportRequest.newBuilder() to construct.
-    private RegionServerReportRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionServerReportRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionServerReportRequest() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionServerReportRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2048,22 +2048,22 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2149,7 +2149,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServer());
@@ -2166,11 +2166,11 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServer());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getLoad());
       }
       size += unknownFields.getSerializedSize();
@@ -2225,61 +2225,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2297,7 +2297,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2305,15 +2305,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.RegionServerReportRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionServerReportRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2326,12 +2326,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerFieldBuilder();
           getLoadFieldBuilder();
@@ -2354,7 +2354,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportRequest_descriptor;
       }
@@ -2400,29 +2400,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest)other);
         } else {
@@ -2460,13 +2460,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2479,7 +2479,7 @@ public final class RegionServerStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
        * required .hbase.pb.ServerName server = 1;
@@ -2582,11 +2582,11 @@ public final class RegionServerStatusProtos {
       /**
        * required .hbase.pb.ServerName server = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerFieldBuilder() {
         if (serverBuilder_ == null) {
-          serverBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServer(),
                   getParentForChildren(),
@@ -2597,7 +2597,7 @@ public final class RegionServerStatusProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad load_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder> loadBuilder_;
       /**
        * 
@@ -2736,11 +2736,11 @@ public final class RegionServerStatusProtos {
        *
        * optional .hbase.pb.ServerLoad load = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder> 
           getLoadFieldBuilder() {
         if (loadBuilder_ == null) {
-          loadBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          loadBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.ServerLoadOrBuilder>(
                   getLoad(),
                   getParentForChildren(),
@@ -2750,12 +2750,12 @@ public final class RegionServerStatusProtos {
         return loadBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2773,22 +2773,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionServerReportRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionServerReportRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2800,34 +2800,34 @@ public final class RegionServerStatusProtos {
 
   public interface RegionServerReportResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionServerReportResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.RegionServerReportResponse}
    */
   public  static final class RegionServerReportResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionServerReportResponse)
       RegionServerReportResponseOrBuilder {
     // Use RegionServerReportResponse.newBuilder() to construct.
-    private RegionServerReportResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionServerReportResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionServerReportResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionServerReportResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2845,22 +2845,22 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2877,7 +2877,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -2921,61 +2921,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2993,7 +2993,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3001,15 +3001,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.RegionServerReportResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionServerReportResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3022,12 +3022,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3036,7 +3036,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionServerReportResponse_descriptor;
       }
@@ -3063,29 +3063,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse)other);
         } else {
@@ -3106,13 +3106,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3123,12 +3123,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3146,22 +3146,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionServerReportResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionServerReportResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3173,7 +3173,7 @@ public final class RegionServerStatusProtos {
 
   public interface ReportRSFatalErrorRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReportRSFatalErrorRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -3223,18 +3223,18 @@ public final class RegionServerStatusProtos {
      *
      * required string error_message = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getErrorMessageBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.ReportRSFatalErrorRequest}
    */
   public  static final class ReportRSFatalErrorRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReportRSFatalErrorRequest)
       ReportRSFatalErrorRequestOrBuilder {
     // Use ReportRSFatalErrorRequest.newBuilder() to construct.
-    private ReportRSFatalErrorRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReportRSFatalErrorRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReportRSFatalErrorRequest() {
@@ -3242,18 +3242,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReportRSFatalErrorRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3283,29 +3283,29 @@ public final class RegionServerStatusProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               errorMessage_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3370,8 +3370,8 @@ public final class RegionServerStatusProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           errorMessage_ = s;
@@ -3386,17 +3386,17 @@ public final class RegionServerStatusProtos {
      *
      * required string error_message = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getErrorMessageBytes() {
       java.lang.Object ref = errorMessage_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         errorMessage_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -3422,13 +3422,13 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServer());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, errorMessage_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, errorMessage_);
       }
       unknownFields.writeTo(output);
     }
@@ -3439,11 +3439,11 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServer());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, errorMessage_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, errorMessage_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -3497,61 +3497,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3569,7 +3569,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3577,15 +3577,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.ReportRSFatalErrorRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReportRSFatalErrorRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3598,12 +3598,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerFieldBuilder();
         }
@@ -3621,7 +3621,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorRequest_descriptor;
       }
@@ -3663,29 +3663,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest)other);
         } else {
@@ -3723,13 +3723,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3742,7 +3742,7 @@ public final class RegionServerStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
        * 
@@ -3881,11 +3881,11 @@ public final class RegionServerStatusProtos {
        *
        * required .hbase.pb.ServerName server = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerFieldBuilder() {
         if (serverBuilder_ == null) {
-          serverBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServer(),
                   getParentForChildren(),
@@ -3916,8 +3916,8 @@ public final class RegionServerStatusProtos {
       public java.lang.String getErrorMessage() {
         java.lang.Object ref = errorMessage_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             errorMessage_ = s;
@@ -3934,17 +3934,17 @@ public final class RegionServerStatusProtos {
        *
        * required string error_message = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getErrorMessageBytes() {
         java.lang.Object ref = errorMessage_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           errorMessage_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -3985,7 +3985,7 @@ public final class RegionServerStatusProtos {
        * required string error_message = 2;
        */
       public Builder setErrorMessageBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3995,12 +3995,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4018,22 +4018,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReportRSFatalErrorRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReportRSFatalErrorRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4045,34 +4045,34 @@ public final class RegionServerStatusProtos {
 
   public interface ReportRSFatalErrorResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReportRSFatalErrorResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * Protobuf type {@code hbase.pb.ReportRSFatalErrorResponse}
    */
   public  static final class ReportRSFatalErrorResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReportRSFatalErrorResponse)
       ReportRSFatalErrorResponseOrBuilder {
     // Use ReportRSFatalErrorResponse.newBuilder() to construct.
-    private ReportRSFatalErrorResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReportRSFatalErrorResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReportRSFatalErrorResponse() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReportRSFatalErrorResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4090,22 +4090,22 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4122,7 +4122,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -4166,61 +4166,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4238,7 +4238,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4246,15 +4246,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.ReportRSFatalErrorResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReportRSFatalErrorResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4267,12 +4267,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -4281,7 +4281,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRSFatalErrorResponse_descriptor;
       }
@@ -4308,29 +4308,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse)other);
         } else {
@@ -4351,13 +4351,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4368,12 +4368,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4391,22 +4391,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReportRSFatalErrorResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReportRSFatalErrorResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4418,7 +4418,7 @@ public final class RegionServerStatusProtos {
 
   public interface GetLastFlushedSequenceIdRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetLastFlushedSequenceIdRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -4435,36 +4435,36 @@ public final class RegionServerStatusProtos {
      *
      * required bytes region_name = 1;
      */
-    com.google.protobuf.ByteString getRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName();
   }
   /**
    * Protobuf type {@code hbase.pb.GetLastFlushedSequenceIdRequest}
    */
   public  static final class GetLastFlushedSequenceIdRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetLastFlushedSequenceIdRequest)
       GetLastFlushedSequenceIdRequestOrBuilder {
     // Use GetLastFlushedSequenceIdRequest.newBuilder() to construct.
-    private GetLastFlushedSequenceIdRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetLastFlushedSequenceIdRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetLastFlushedSequenceIdRequest() {
-      regionName_ = com.google.protobuf.ByteString.EMPTY;
+      regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetLastFlushedSequenceIdRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4487,22 +4487,22 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4511,7 +4511,7 @@ public final class RegionServerStatusProtos {
 
     private int bitField0_;
     public static final int REGION_NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString regionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_;
     /**
      * 
      ** region name 
@@ -4529,7 +4529,7 @@ public final class RegionServerStatusProtos {
      *
      * required bytes region_name = 1;
      */
-    public com.google.protobuf.ByteString getRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
       return regionName_;
     }
 
@@ -4547,7 +4547,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, regionName_);
@@ -4561,7 +4561,7 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, regionName_);
       }
       size += unknownFields.getSerializedSize();
@@ -4607,61 +4607,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4679,7 +4679,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4687,15 +4687,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.GetLastFlushedSequenceIdRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetLastFlushedSequenceIdRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4708,23 +4708,23 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_descriptor;
       }
@@ -4758,29 +4758,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest)other);
         } else {
@@ -4807,13 +4807,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4825,7 +4825,7 @@ public final class RegionServerStatusProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        ** region name 
@@ -4843,7 +4843,7 @@ public final class RegionServerStatusProtos {
        *
        * required bytes region_name = 1;
        */
-      public com.google.protobuf.ByteString getRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
         return regionName_;
       }
       /**
@@ -4853,7 +4853,7 @@ public final class RegionServerStatusProtos {
        *
        * required bytes region_name = 1;
        */
-      public Builder setRegionName(com.google.protobuf.ByteString value) {
+      public Builder setRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4876,12 +4876,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4899,22 +4899,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetLastFlushedSequenceIdRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetLastFlushedSequenceIdRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4926,7 +4926,7 @@ public final class RegionServerStatusProtos {
 
   public interface GetLastFlushedSequenceIdResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.GetLastFlushedSequenceIdResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -4993,11 +4993,11 @@ public final class RegionServerStatusProtos {
    * Protobuf type {@code hbase.pb.GetLastFlushedSequenceIdResponse}
    */
   public  static final class GetLastFlushedSequenceIdResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.GetLastFlushedSequenceIdResponse)
       GetLastFlushedSequenceIdResponseOrBuilder {
     // Use GetLastFlushedSequenceIdResponse.newBuilder() to construct.
-    private GetLastFlushedSequenceIdResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private GetLastFlushedSequenceIdResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private GetLastFlushedSequenceIdResponse() {
@@ -5006,18 +5006,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private GetLastFlushedSequenceIdResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5049,10 +5049,10 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -5062,12 +5062,12 @@ public final class RegionServerStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5173,7 +5173,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeUInt64(1, lastFlushedSequenceId_);
@@ -5190,11 +5190,11 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(1, lastFlushedSequenceId_);
       }
       for (int i = 0; i < storeLastFlushedSequenceId_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, storeLastFlushedSequenceId_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -5234,7 +5234,7 @@ public final class RegionServerStatusProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasLastFlushedSequenceId()) {
         hash = (37 * hash) + LAST_FLUSHED_SEQUENCE_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLastFlushedSequenceId());
       }
       if (getStoreLastFlushedSequenceIdCount() > 0) {
@@ -5247,61 +5247,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5319,7 +5319,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5327,15 +5327,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.GetLastFlushedSequenceIdResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.GetLastFlushedSequenceIdResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5348,12 +5348,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getStoreLastFlushedSequenceIdFieldBuilder();
         }
@@ -5371,7 +5371,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_descriptor;
       }
@@ -5414,29 +5414,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse)other);
         } else {
@@ -5469,7 +5469,7 @@ public final class RegionServerStatusProtos {
               storeLastFlushedSequenceId_ = other.storeLastFlushedSequenceId_;
               bitField0_ = (bitField0_ & ~0x00000002);
               storeLastFlushedSequenceIdBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStoreLastFlushedSequenceIdFieldBuilder() : null;
             } else {
               storeLastFlushedSequenceIdBuilder_.addAllMessages(other.storeLastFlushedSequenceId_);
@@ -5494,13 +5494,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5569,7 +5569,7 @@ public final class RegionServerStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder> storeLastFlushedSequenceIdBuilder_;
 
       /**
@@ -5741,7 +5741,7 @@ public final class RegionServerStatusProtos {
           java.lang.Iterable values) {
         if (storeLastFlushedSequenceIdBuilder_ == null) {
           ensureStoreLastFlushedSequenceIdIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, storeLastFlushedSequenceId_);
           onChanged();
         } else {
@@ -5857,11 +5857,11 @@ public final class RegionServerStatusProtos {
            getStoreLastFlushedSequenceIdBuilderList() {
         return getStoreLastFlushedSequenceIdFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder> 
           getStoreLastFlushedSequenceIdFieldBuilder() {
         if (storeLastFlushedSequenceIdBuilder_ == null) {
-          storeLastFlushedSequenceIdBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          storeLastFlushedSequenceIdBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder>(
                   storeLastFlushedSequenceId_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -5872,12 +5872,12 @@ public final class RegionServerStatusProtos {
         return storeLastFlushedSequenceIdBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5895,22 +5895,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public GetLastFlushedSequenceIdResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new GetLastFlushedSequenceIdResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5922,7 +5922,7 @@ public final class RegionServerStatusProtos {
 
   public interface RegionStateTransitionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionStateTransition)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionStateTransition.TransitionCode transition_code = 1;
@@ -5998,11 +5998,11 @@ public final class RegionServerStatusProtos {
    * Protobuf type {@code hbase.pb.RegionStateTransition}
    */
   public  static final class RegionStateTransition extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionStateTransition)
       RegionStateTransitionOrBuilder {
     // Use RegionStateTransition.newBuilder() to construct.
-    private RegionStateTransition(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionStateTransition(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionStateTransition() {
@@ -6012,18 +6012,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionStateTransition(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -6066,10 +6066,10 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -6079,12 +6079,12 @@ public final class RegionServerStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionStateTransition_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionStateTransition_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -6095,7 +6095,7 @@ public final class RegionServerStatusProtos {
      * Protobuf enum {@code hbase.pb.RegionStateTransition.TransitionCode}
      */
     public enum TransitionCode
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * OPENED = 0;
        */
@@ -6233,27 +6233,27 @@ public final class RegionServerStatusProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           TransitionCode> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public TransitionCode findValueByNumber(int number) {
                 return TransitionCode.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.getDescriptor().getEnumTypes().get(0);
       }
@@ -6261,7 +6261,7 @@ public final class RegionServerStatusProtos {
       private static final TransitionCode[] VALUES = values();
 
       public static TransitionCode valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -6393,7 +6393,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, transitionCode_);
@@ -6413,15 +6413,15 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, transitionCode_);
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, regionInfo_.get(i));
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, openSeqNum_);
       }
       size += unknownFields.getSerializedSize();
@@ -6473,7 +6473,7 @@ public final class RegionServerStatusProtos {
       }
       if (hasOpenSeqNum()) {
         hash = (37 * hash) + OPEN_SEQ_NUM_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getOpenSeqNum());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -6482,61 +6482,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -6554,7 +6554,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -6562,15 +6562,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.RegionStateTransition}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionStateTransition)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransitionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionStateTransition_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionStateTransition_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -6583,12 +6583,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionInfoFieldBuilder();
         }
@@ -6608,7 +6608,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_RegionStateTransition_descriptor;
       }
@@ -6655,29 +6655,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition)other);
         } else {
@@ -6710,7 +6710,7 @@ public final class RegionServerStatusProtos {
               regionInfo_ = other.regionInfo_;
               bitField0_ = (bitField0_ & ~0x00000002);
               regionInfoBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
             } else {
               regionInfoBuilder_.addAllMessages(other.regionInfo_);
@@ -6738,13 +6738,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -6801,7 +6801,7 @@ public final class RegionServerStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
@@ -6973,7 +6973,7 @@ public final class RegionServerStatusProtos {
           java.lang.Iterable values) {
         if (regionInfoBuilder_ == null) {
           ensureRegionInfoIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionInfo_);
           onChanged();
         } else {
@@ -7089,11 +7089,11 @@ public final class RegionServerStatusProtos {
            getRegionInfoBuilderList() {
         return getRegionInfoFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   regionInfo_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -7152,12 +7152,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7175,22 +7175,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionStateTransition parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionStateTransition(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7202,7 +7202,7 @@ public final class RegionServerStatusProtos {
 
   public interface ReportRegionStateTransitionRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReportRegionStateTransitionRequest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -7257,11 +7257,11 @@ public final class RegionServerStatusProtos {
    * Protobuf type {@code hbase.pb.ReportRegionStateTransitionRequest}
    */
   public  static final class ReportRegionStateTransitionRequest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReportRegionStateTransitionRequest)
       ReportRegionStateTransitionRequestOrBuilder {
     // Use ReportRegionStateTransitionRequest.newBuilder() to construct.
-    private ReportRegionStateTransitionRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReportRegionStateTransitionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReportRegionStateTransitionRequest() {
@@ -7269,18 +7269,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReportRegionStateTransitionRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7320,10 +7320,10 @@ public final class RegionServerStatusProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -7333,12 +7333,12 @@ public final class RegionServerStatusProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionRequest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7438,7 +7438,7 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServer());
@@ -7455,11 +7455,11 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServer());
       }
       for (int i = 0; i < transition_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, transition_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -7511,61 +7511,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7583,7 +7583,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7591,15 +7591,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.ReportRegionStateTransitionRequest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReportRegionStateTransitionRequest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionRequest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7612,12 +7612,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerFieldBuilder();
           getTransitionFieldBuilder();
@@ -7640,7 +7640,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionRequest_descriptor;
       }
@@ -7687,29 +7687,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest)other);
         } else {
@@ -7742,7 +7742,7 @@ public final class RegionServerStatusProtos {
               transition_ = other.transition_;
               bitField0_ = (bitField0_ & ~0x00000002);
               transitionBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTransitionFieldBuilder() : null;
             } else {
               transitionBuilder_.addAllMessages(other.transition_);
@@ -7770,13 +7770,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7789,7 +7789,7 @@ public final class RegionServerStatusProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
        * 
@@ -7928,11 +7928,11 @@ public final class RegionServerStatusProtos {
        *
        * required .hbase.pb.ServerName server = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerFieldBuilder() {
         if (serverBuilder_ == null) {
-          serverBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServer(),
                   getParentForChildren(),
@@ -7951,7 +7951,7 @@ public final class RegionServerStatusProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransitionOrBuilder> transitionBuilder_;
 
       /**
@@ -8083,7 +8083,7 @@ public final class RegionServerStatusProtos {
           java.lang.Iterable values) {
         if (transitionBuilder_ == null) {
           ensureTransitionIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, transition_);
           onChanged();
         } else {
@@ -8167,11 +8167,11 @@ public final class RegionServerStatusProtos {
            getTransitionBuilderList() {
         return getTransitionFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransitionOrBuilder> 
           getTransitionFieldBuilder() {
         if (transitionBuilder_ == null) {
-          transitionBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          transitionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransitionOrBuilder>(
                   transition_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -8182,12 +8182,12 @@ public final class RegionServerStatusProtos {
         return transitionBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8205,22 +8205,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReportRegionStateTransitionRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReportRegionStateTransitionRequest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8232,7 +8232,7 @@ public final class RegionServerStatusProtos {
 
   public interface ReportRegionStateTransitionResponseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReportRegionStateTransitionResponse)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -8257,18 +8257,18 @@ public final class RegionServerStatusProtos {
      *
      * optional string error_message = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getErrorMessageBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.ReportRegionStateTransitionResponse}
    */
   public  static final class ReportRegionStateTransitionResponse extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReportRegionStateTransitionResponse)
       ReportRegionStateTransitionResponseOrBuilder {
     // Use ReportRegionStateTransitionResponse.newBuilder() to construct.
-    private ReportRegionStateTransitionResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReportRegionStateTransitionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReportRegionStateTransitionResponse() {
@@ -8276,18 +8276,18 @@ public final class RegionServerStatusProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReportRegionStateTransitionResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8304,29 +8304,29 @@ public final class RegionServerStatusProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               errorMessage_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8358,8 +8358,8 @@ public final class RegionServerStatusProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           errorMessage_ = s;
@@ -8374,17 +8374,17 @@ public final class RegionServerStatusProtos {
      *
      * optional string error_message = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getErrorMessageBytes() {
       java.lang.Object ref = errorMessage_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         errorMessage_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -8398,10 +8398,10 @@ public final class RegionServerStatusProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, errorMessage_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, errorMessage_);
       }
       unknownFields.writeTo(output);
     }
@@ -8412,7 +8412,7 @@ public final class RegionServerStatusProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, errorMessage_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, errorMessage_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -8457,61 +8457,61 @@ public final class RegionServerStatusProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8529,7 +8529,7 @@ public final class RegionServerStatusProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8537,15 +8537,15 @@ public final class RegionServerStatusProtos {
      * Protobuf type {@code hbase.pb.ReportRegionStateTransitionResponse}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReportRegionStateTransitionResponse)
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8558,12 +8558,12 @@ public final class RegionServerStatusProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -8574,7 +8574,7 @@ public final class RegionServerStatusProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor;
       }
@@ -8608,29 +8608,29 @@ public final class RegionServerStatusProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse)other);
         } else {
@@ -8656,13 +8656,13 @@ public final class RegionServerStatusProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8695,8 +8695,8 @@ public final class RegionServerStatusProtos {
       public java.lang.String getErrorMessage() {
         java.lang.Object ref = errorMessage_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             errorMessage_ = s;
@@ -8713,17 +8713,17 @@ public final class RegionServerStatusProtos {
        *
        * optional string error_message = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getErrorMessageBytes() {
         java.lang.Object ref = errorMessage_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           errorMessage_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -8764,7 +8764,7 @@ public final class RegionServerStatusProtos {
        * optional string error_message = 1;
        */
       public Builder setErrorMessageBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8774,12 +8774,12 @@ public final class RegionServerStatusProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8797,22 +8797,22 @@ public final class RegionServerStatusProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReportRegionStateTransitionResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReportRegionStateTransitionResponse(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8826,7 +8826,7 @@ public final class RegionServerStatusProtos {
    * Protobuf service {@code hbase.pb.RegionServerStatusService}
    */
   public static abstract class RegionServerStatusService
-      implements com.google.protobuf.Service {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.Service {
     protected RegionServerStatusService() {}
 
     public interface Interface {
@@ -8838,9 +8838,9 @@ public final class RegionServerStatusProtos {
        * rpc RegionServerStartup(.hbase.pb.RegionServerStartupRequest) returns (.hbase.pb.RegionServerStartupResponse);
        */
       public abstract void regionServerStartup(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -8850,9 +8850,9 @@ public final class RegionServerStatusProtos {
        * rpc RegionServerReport(.hbase.pb.RegionServerReportRequest) returns (.hbase.pb.RegionServerReportResponse);
        */
       public abstract void regionServerReport(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -8864,9 +8864,9 @@ public final class RegionServerStatusProtos {
        * rpc ReportRSFatalError(.hbase.pb.ReportRSFatalErrorRequest) returns (.hbase.pb.ReportRSFatalErrorResponse);
        */
       public abstract void reportRSFatalError(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -8878,9 +8878,9 @@ public final class RegionServerStatusProtos {
        * rpc GetLastFlushedSequenceId(.hbase.pb.GetLastFlushedSequenceIdRequest) returns (.hbase.pb.GetLastFlushedSequenceIdResponse);
        */
       public abstract void getLastFlushedSequenceId(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
       /**
        * 
@@ -8893,71 +8893,71 @@ public final class RegionServerStatusProtos {
        * rpc ReportRegionStateTransition(.hbase.pb.ReportRegionStateTransitionRequest) returns (.hbase.pb.ReportRegionStateTransitionResponse);
        */
       public abstract void reportRegionStateTransition(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request,
-          com.google.protobuf.RpcCallback done);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     }
 
-    public static com.google.protobuf.Service newReflectiveService(
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
         final Interface impl) {
       return new RegionServerStatusService() {
         @java.lang.Override
         public  void regionServerStartup(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.regionServerStartup(controller, request, done);
         }
 
         @java.lang.Override
         public  void regionServerReport(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.regionServerReport(controller, request, done);
         }
 
         @java.lang.Override
         public  void reportRSFatalError(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.reportRSFatalError(controller, request, done);
         }
 
         @java.lang.Override
         public  void getLastFlushedSequenceId(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.getLastFlushedSequenceId(controller, request, done);
         }
 
         @java.lang.Override
         public  void reportRegionStateTransition(
-            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request,
-            com.google.protobuf.RpcCallback done) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
           impl.reportRegionStateTransition(controller, request, done);
         }
 
       };
     }
 
-    public static com.google.protobuf.BlockingService
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
         newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
+      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService() {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
             getDescriptorForType() {
           return getDescriptor();
         }
 
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message callBlockingMethod(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.callBlockingMethod() given method descriptor for " +
@@ -8979,9 +8979,9 @@ public final class RegionServerStatusProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getRequestPrototype() given method " +
@@ -9003,9 +9003,9 @@ public final class RegionServerStatusProtos {
           }
         }
 
-        public final com.google.protobuf.Message
+        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
             getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
           if (method.getService() != getDescriptor()) {
             throw new java.lang.IllegalArgumentException(
               "Service.getResponsePrototype() given method " +
@@ -9038,9 +9038,9 @@ public final class RegionServerStatusProtos {
      * rpc RegionServerStartup(.hbase.pb.RegionServerStartupRequest) returns (.hbase.pb.RegionServerStartupResponse);
      */
     public abstract void regionServerStartup(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -9050,9 +9050,9 @@ public final class RegionServerStatusProtos {
      * rpc RegionServerReport(.hbase.pb.RegionServerReportRequest) returns (.hbase.pb.RegionServerReportResponse);
      */
     public abstract void regionServerReport(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -9064,9 +9064,9 @@ public final class RegionServerStatusProtos {
      * rpc ReportRSFatalError(.hbase.pb.ReportRSFatalErrorRequest) returns (.hbase.pb.ReportRSFatalErrorResponse);
      */
     public abstract void reportRSFatalError(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -9078,9 +9078,9 @@ public final class RegionServerStatusProtos {
      * rpc GetLastFlushedSequenceId(.hbase.pb.GetLastFlushedSequenceIdRequest) returns (.hbase.pb.GetLastFlushedSequenceIdResponse);
      */
     public abstract void getLastFlushedSequenceId(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     /**
      * 
@@ -9093,26 +9093,26 @@ public final class RegionServerStatusProtos {
      * rpc ReportRegionStateTransition(.hbase.pb.ReportRegionStateTransitionRequest) returns (.hbase.pb.ReportRegionStateTransitionResponse);
      */
     public abstract void reportRegionStateTransition(
-        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request,
-        com.google.protobuf.RpcCallback done);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done);
 
     public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.getDescriptor().getServices().get(0);
     }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
 
     public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Message request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Message> done) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.callMethod() given method descriptor for wrong " +
@@ -9121,27 +9121,27 @@ public final class RegionServerStatusProtos {
       switch(method.getIndex()) {
         case 0:
           this.regionServerStartup(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 1:
           this.regionServerReport(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 2:
           this.reportRSFatalError(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 3:
           this.getLastFlushedSequenceId(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         case 4:
           this.reportRegionStateTransition(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest)request,
-            com.google.protobuf.RpcUtil.specializeCallback(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback(
               done));
           return;
         default:
@@ -9149,9 +9149,9 @@ public final class RegionServerStatusProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getRequestPrototype() given method " +
@@ -9173,9 +9173,9 @@ public final class RegionServerStatusProtos {
       }
     }
 
-    public final com.google.protobuf.Message
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
         getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.MethodDescriptor method) {
       if (method.getService() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "Service.getResponsePrototype() given method " +
@@ -9198,91 +9198,91 @@ public final class RegionServerStatusProtos {
     }
 
     public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
       return new Stub(channel);
     }
 
     public static final class Stub extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
+      private Stub(org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.RpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel channel;
 
-      public com.google.protobuf.RpcChannel getChannel() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel getChannel() {
         return channel;
       }
 
       public  void regionServerStartup(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(0),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.getDefaultInstance()));
       }
 
       public  void regionServerReport(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(1),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse.getDefaultInstance()));
       }
 
       public  void reportRSFatalError(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(2),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse.getDefaultInstance()));
       }
 
       public  void getLastFlushedSequenceId(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(3),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse.getDefaultInstance()));
       }
 
       public  void reportRegionStateTransition(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request,
-          com.google.protobuf.RpcCallback done) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) {
         channel.callMethod(
           getDescriptor().getMethods().get(4),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
             done,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.getDefaultInstance()));
@@ -9290,48 +9290,48 @@ public final class RegionServerStatusProtos {
     }
 
     public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
       return new BlockingStub(channel);
     }
 
     public interface BlockingInterface {
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse regionServerStartup(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse regionServerReport(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse reportRSFatalError(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse reportRegionStateTransition(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request)
-          throws com.google.protobuf.ServiceException;
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+      private BlockingStub(org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel) {
         this.channel = channel;
       }
 
-      private final com.google.protobuf.BlockingRpcChannel channel;
+      private final org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel channel;
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse regionServerStartup(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(0),
           controller,
@@ -9341,9 +9341,9 @@ public final class RegionServerStatusProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse regionServerReport(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(1),
           controller,
@@ -9353,9 +9353,9 @@ public final class RegionServerStatusProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse reportRSFatalError(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(2),
           controller,
@@ -9365,9 +9365,9 @@ public final class RegionServerStatusProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(3),
           controller,
@@ -9377,9 +9377,9 @@ public final class RegionServerStatusProtos {
 
 
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse reportRegionStateTransition(
-          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request)
-          throws com.google.protobuf.ServiceException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(4),
           controller,
@@ -9392,67 +9392,67 @@ public final class RegionServerStatusProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.RegionServerStatusService)
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionServerStartupRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionServerStartupRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionServerStartupResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionServerStartupResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionServerReportRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionServerReportRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionServerReportResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionServerReportResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReportRSFatalErrorRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReportRSFatalErrorRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReportRSFatalErrorResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReportRSFatalErrorResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionStateTransition_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionStateTransition_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReportRegionStateTransitionRequest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReportRegionStateTransitionRequest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReportRegionStateTransitionResponse_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -9506,84 +9506,84 @@ public final class RegionServerStatusProtos {
       "rotobuf.generatedB\030RegionServerStatusPro" +
       "tosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_RegionServerStartupRequest_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_RegionServerStartupRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionServerStartupRequest_descriptor,
         new java.lang.String[] { "Port", "ServerStartCode", "ServerCurrentTime", "UseThisHostnameInstead", });
     internal_static_hbase_pb_RegionServerStartupResponse_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_RegionServerStartupResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionServerStartupResponse_descriptor,
         new java.lang.String[] { "MapEntries", });
     internal_static_hbase_pb_RegionServerReportRequest_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_RegionServerReportRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionServerReportRequest_descriptor,
         new java.lang.String[] { "Server", "Load", });
     internal_static_hbase_pb_RegionServerReportResponse_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_RegionServerReportResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionServerReportResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_ReportRSFatalErrorRequest_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_ReportRSFatalErrorRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReportRSFatalErrorRequest_descriptor,
         new java.lang.String[] { "Server", "ErrorMessage", });
     internal_static_hbase_pb_ReportRSFatalErrorResponse_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_ReportRSFatalErrorResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReportRSFatalErrorResponse_descriptor,
         new java.lang.String[] { });
     internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetLastFlushedSequenceIdRequest_descriptor,
         new java.lang.String[] { "RegionName", });
     internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetLastFlushedSequenceIdResponse_descriptor,
         new java.lang.String[] { "LastFlushedSequenceId", "StoreLastFlushedSequenceId", });
     internal_static_hbase_pb_RegionStateTransition_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_RegionStateTransition_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionStateTransition_descriptor,
         new java.lang.String[] { "TransitionCode", "RegionInfo", "OpenSeqNum", });
     internal_static_hbase_pb_ReportRegionStateTransitionRequest_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_ReportRegionStateTransitionRequest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReportRegionStateTransitionRequest_descriptor,
         new java.lang.String[] { "Server", "Transition", });
     internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_ReportRegionStateTransitionResponse_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor,
         new java.lang.String[] { "ErrorMessage", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
index 4d3b294..d40086b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class SnapshotProtos {
   private SnapshotProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface SnapshotFileInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotFileInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.SnapshotFileInfo.Type type = 1;
@@ -38,7 +38,7 @@ public final class SnapshotProtos {
     /**
      * optional string hfile = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getHfileBytes();
 
     /**
@@ -52,7 +52,7 @@ public final class SnapshotProtos {
     /**
      * optional string wal_server = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getWalServerBytes();
 
     /**
@@ -66,18 +66,18 @@ public final class SnapshotProtos {
     /**
      * optional string wal_name = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getWalNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.SnapshotFileInfo}
    */
   public  static final class SnapshotFileInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotFileInfo)
       SnapshotFileInfoOrBuilder {
     // Use SnapshotFileInfo.newBuilder() to construct.
-    private SnapshotFileInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SnapshotFileInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SnapshotFileInfo() {
@@ -88,18 +88,18 @@ public final class SnapshotProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SnapshotFileInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -127,41 +127,41 @@ public final class SnapshotProtos {
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               hfile_ = bs;
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000004;
               walServer_ = bs;
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               walName_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -172,7 +172,7 @@ public final class SnapshotProtos {
      * Protobuf enum {@code hbase.pb.SnapshotFileInfo.Type}
      */
     public enum Type
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * HFILE = 1;
        */
@@ -213,27 +213,27 @@ public final class SnapshotProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           Type> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public Type findValueByNumber(int number) {
                 return Type.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo.getDescriptor().getEnumTypes().get(0);
       }
@@ -241,7 +241,7 @@ public final class SnapshotProtos {
       private static final Type[] VALUES = values();
 
       public static Type valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -291,8 +291,8 @@ public final class SnapshotProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           hfile_ = s;
@@ -303,17 +303,17 @@ public final class SnapshotProtos {
     /**
      * optional string hfile = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getHfileBytes() {
       java.lang.Object ref = hfile_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         hfile_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -333,8 +333,8 @@ public final class SnapshotProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           walServer_ = s;
@@ -345,17 +345,17 @@ public final class SnapshotProtos {
     /**
      * optional string wal_server = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getWalServerBytes() {
       java.lang.Object ref = walServer_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         walServer_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -375,8 +375,8 @@ public final class SnapshotProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           walName_ = s;
@@ -387,17 +387,17 @@ public final class SnapshotProtos {
     /**
      * optional string wal_name = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getWalNameBytes() {
       java.lang.Object ref = walName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         walName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -415,19 +415,19 @@ public final class SnapshotProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, type_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, hfile_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, hfile_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, walServer_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, walServer_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, walName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, walName_);
       }
       unknownFields.writeTo(output);
     }
@@ -438,17 +438,17 @@ public final class SnapshotProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, type_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, hfile_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(3, hfile_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, walServer_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, walServer_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, walName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, walName_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -519,61 +519,61 @@ public final class SnapshotProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -591,7 +591,7 @@ public final class SnapshotProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -599,15 +599,15 @@ public final class SnapshotProtos {
      * Protobuf type {@code hbase.pb.SnapshotFileInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotFileInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -620,12 +620,12 @@ public final class SnapshotProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -642,7 +642,7 @@ public final class SnapshotProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
       }
@@ -688,29 +688,29 @@ public final class SnapshotProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo)other);
         } else {
@@ -752,13 +752,13 @@ public final class SnapshotProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -819,8 +819,8 @@ public final class SnapshotProtos {
       public java.lang.String getHfile() {
         java.lang.Object ref = hfile_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             hfile_ = s;
@@ -833,17 +833,17 @@ public final class SnapshotProtos {
       /**
        * optional string hfile = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getHfileBytes() {
         java.lang.Object ref = hfile_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           hfile_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -872,7 +872,7 @@ public final class SnapshotProtos {
        * optional string hfile = 3;
        */
       public Builder setHfileBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -895,8 +895,8 @@ public final class SnapshotProtos {
       public java.lang.String getWalServer() {
         java.lang.Object ref = walServer_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             walServer_ = s;
@@ -909,17 +909,17 @@ public final class SnapshotProtos {
       /**
        * optional string wal_server = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getWalServerBytes() {
         java.lang.Object ref = walServer_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           walServer_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -948,7 +948,7 @@ public final class SnapshotProtos {
        * optional string wal_server = 4;
        */
       public Builder setWalServerBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -971,8 +971,8 @@ public final class SnapshotProtos {
       public java.lang.String getWalName() {
         java.lang.Object ref = walName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             walName_ = s;
@@ -985,17 +985,17 @@ public final class SnapshotProtos {
       /**
        * optional string wal_name = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getWalNameBytes() {
         java.lang.Object ref = walName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           walName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1024,7 +1024,7 @@ public final class SnapshotProtos {
        * optional string wal_name = 5;
        */
       public Builder setWalNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1034,12 +1034,12 @@ public final class SnapshotProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1057,22 +1057,22 @@ public final class SnapshotProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SnapshotFileInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SnapshotFileInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1084,7 +1084,7 @@ public final class SnapshotProtos {
 
   public interface SnapshotRegionManifestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotRegionManifest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional int32 version = 1;
@@ -1136,11 +1136,11 @@ public final class SnapshotProtos {
    * Protobuf type {@code hbase.pb.SnapshotRegionManifest}
    */
   public  static final class SnapshotRegionManifest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotRegionManifest)
       SnapshotRegionManifestOrBuilder {
     // Use SnapshotRegionManifest.newBuilder() to construct.
-    private SnapshotRegionManifest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SnapshotRegionManifest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SnapshotRegionManifest() {
@@ -1149,18 +1149,18 @@ public final class SnapshotProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SnapshotRegionManifest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1205,10 +1205,10 @@ public final class SnapshotProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -1218,12 +1218,12 @@ public final class SnapshotProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1232,7 +1232,7 @@ public final class SnapshotProtos {
 
     public interface StoreFileOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotRegionManifest.StoreFile)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required string name = 1;
@@ -1245,7 +1245,7 @@ public final class SnapshotProtos {
       /**
        * required string name = 1;
        */
-      com.google.protobuf.ByteString
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes();
 
       /**
@@ -1282,11 +1282,11 @@ public final class SnapshotProtos {
      * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile}
      */
     public  static final class StoreFile extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotRegionManifest.StoreFile)
         StoreFileOrBuilder {
       // Use StoreFile.newBuilder() to construct.
-      private StoreFile(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private StoreFile(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private StoreFile() {
@@ -1295,18 +1295,18 @@ public final class SnapshotProtos {
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private StoreFile(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -1323,7 +1323,7 @@ public final class SnapshotProtos {
                 break;
               }
               case 10: {
-                com.google.protobuf.ByteString bs = input.readBytes();
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
                 bitField0_ |= 0x00000001;
                 name_ = bs;
                 break;
@@ -1348,22 +1348,22 @@ public final class SnapshotProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           this.unknownFields = unknownFields.build();
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1387,8 +1387,8 @@ public final class SnapshotProtos {
         if (ref instanceof java.lang.String) {
           return (java.lang.String) ref;
         } else {
-          com.google.protobuf.ByteString bs = 
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             name_ = s;
@@ -1399,17 +1399,17 @@ public final class SnapshotProtos {
       /**
        * required string name = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof java.lang.String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
 
@@ -1477,10 +1477,10 @@ public final class SnapshotProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_);
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
           output.writeMessage(2, getReference());
@@ -1497,14 +1497,14 @@ public final class SnapshotProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_);
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(2, getReference());
         }
         if (((bitField0_ & 0x00000004) == 0x00000004)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeUInt64Size(3, fileSize_);
         }
         size += unknownFields.getSerializedSize();
@@ -1560,7 +1560,7 @@ public final class SnapshotProtos {
         }
         if (hasFileSize()) {
           hash = (37 * hash) + FILE_SIZE_FIELD_NUMBER;
-          hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+          hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
               getFileSize());
         }
         hash = (29 * hash) + unknownFields.hashCode();
@@ -1569,61 +1569,61 @@ public final class SnapshotProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -1641,7 +1641,7 @@ public final class SnapshotProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -1649,15 +1649,15 @@ public final class SnapshotProtos {
        * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotRegionManifest.StoreFile)
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -1670,12 +1670,12 @@ public final class SnapshotProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
             getReferenceFieldBuilder();
           }
@@ -1695,7 +1695,7 @@ public final class SnapshotProtos {
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
         }
@@ -1741,29 +1741,29 @@ public final class SnapshotProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile)other);
           } else {
@@ -1803,13 +1803,13 @@ public final class SnapshotProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -1834,8 +1834,8 @@ public final class SnapshotProtos {
         public java.lang.String getName() {
           java.lang.Object ref = name_;
           if (!(ref instanceof java.lang.String)) {
-            com.google.protobuf.ByteString bs =
-                (com.google.protobuf.ByteString) ref;
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+                (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
             java.lang.String s = bs.toStringUtf8();
             if (bs.isValidUtf8()) {
               name_ = s;
@@ -1848,17 +1848,17 @@ public final class SnapshotProtos {
         /**
          * required string name = 1;
          */
-        public com.google.protobuf.ByteString
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
             getNameBytes() {
           java.lang.Object ref = name_;
           if (ref instanceof String) {
-            com.google.protobuf.ByteString b = 
-                com.google.protobuf.ByteString.copyFromUtf8(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                     (java.lang.String) ref);
             name_ = b;
             return b;
           } else {
-            return (com.google.protobuf.ByteString) ref;
+            return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           }
         }
         /**
@@ -1887,7 +1887,7 @@ public final class SnapshotProtos {
          * required string name = 1;
          */
         public Builder setNameBytes(
-            com.google.protobuf.ByteString value) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -1898,7 +1898,7 @@ public final class SnapshotProtos {
         }
 
         private org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference reference_ = null;
-        private com.google.protobuf.SingleFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.ReferenceOrBuilder> referenceBuilder_;
         /**
          * optional .hbase.pb.Reference reference = 2;
@@ -2001,11 +2001,11 @@ public final class SnapshotProtos {
         /**
          * optional .hbase.pb.Reference reference = 2;
          */
-        private com.google.protobuf.SingleFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.ReferenceOrBuilder> 
             getReferenceFieldBuilder() {
           if (referenceBuilder_ == null) {
-            referenceBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+            referenceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.ReferenceOrBuilder>(
                     getReference(),
                     getParentForChildren(),
@@ -2063,12 +2063,12 @@ public final class SnapshotProtos {
           return this;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -2086,22 +2086,22 @@ public final class SnapshotProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public StoreFile parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new StoreFile(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -2113,7 +2113,7 @@ public final class SnapshotProtos {
 
     public interface FamilyFilesOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotRegionManifest.FamilyFiles)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required bytes family_name = 1;
@@ -2122,7 +2122,7 @@ public final class SnapshotProtos {
       /**
        * required bytes family_name = 1;
        */
-      com.google.protobuf.ByteString getFamilyName();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName();
 
       /**
        * repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
@@ -2152,31 +2152,31 @@ public final class SnapshotProtos {
      * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles}
      */
     public  static final class FamilyFiles extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotRegionManifest.FamilyFiles)
         FamilyFilesOrBuilder {
       // Use FamilyFiles.newBuilder() to construct.
-      private FamilyFiles(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private FamilyFiles(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private FamilyFiles() {
-        familyName_ = com.google.protobuf.ByteString.EMPTY;
+        familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         storeFiles_ = java.util.Collections.emptyList();
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private FamilyFiles(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -2208,10 +2208,10 @@ public final class SnapshotProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -2221,12 +2221,12 @@ public final class SnapshotProtos {
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2235,7 +2235,7 @@ public final class SnapshotProtos {
 
       private int bitField0_;
       public static final int FAMILY_NAME_FIELD_NUMBER = 1;
-      private com.google.protobuf.ByteString familyName_;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_;
       /**
        * required bytes family_name = 1;
        */
@@ -2245,7 +2245,7 @@ public final class SnapshotProtos {
       /**
        * required bytes family_name = 1;
        */
-      public com.google.protobuf.ByteString getFamilyName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
         return familyName_;
       }
 
@@ -2304,7 +2304,7 @@ public final class SnapshotProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeBytes(1, familyName_);
@@ -2321,11 +2321,11 @@ public final class SnapshotProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSize(1, familyName_);
         }
         for (int i = 0; i < storeFiles_.size(); i++) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeMessageSize(2, storeFiles_.get(i));
         }
         size += unknownFields.getSerializedSize();
@@ -2377,61 +2377,61 @@ public final class SnapshotProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -2449,7 +2449,7 @@ public final class SnapshotProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -2457,15 +2457,15 @@ public final class SnapshotProtos {
        * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotRegionManifest.FamilyFiles)
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -2478,19 +2478,19 @@ public final class SnapshotProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
             getStoreFilesFieldBuilder();
           }
         }
         public Builder clear() {
           super.clear();
-          familyName_ = com.google.protobuf.ByteString.EMPTY;
+          familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           bitField0_ = (bitField0_ & ~0x00000001);
           if (storeFilesBuilder_ == null) {
             storeFiles_ = java.util.Collections.emptyList();
@@ -2501,7 +2501,7 @@ public final class SnapshotProtos {
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
         }
@@ -2544,29 +2544,29 @@ public final class SnapshotProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles)other);
           } else {
@@ -2599,7 +2599,7 @@ public final class SnapshotProtos {
                 storeFiles_ = other.storeFiles_;
                 bitField0_ = (bitField0_ & ~0x00000002);
                 storeFilesBuilder_ = 
-                  com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                  org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                      getStoreFilesFieldBuilder() : null;
               } else {
                 storeFilesBuilder_.addAllMessages(other.storeFiles_);
@@ -2624,13 +2624,13 @@ public final class SnapshotProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -2642,7 +2642,7 @@ public final class SnapshotProtos {
         }
         private int bitField0_;
 
-        private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         /**
          * required bytes family_name = 1;
          */
@@ -2652,13 +2652,13 @@ public final class SnapshotProtos {
         /**
          * required bytes family_name = 1;
          */
-        public com.google.protobuf.ByteString getFamilyName() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
           return familyName_;
         }
         /**
          * required bytes family_name = 1;
          */
-        public Builder setFamilyName(com.google.protobuf.ByteString value) {
+        public Builder setFamilyName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -2686,7 +2686,7 @@ public final class SnapshotProtos {
            }
         }
 
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> storeFilesBuilder_;
 
         /**
@@ -2818,7 +2818,7 @@ public final class SnapshotProtos {
             java.lang.Iterable values) {
           if (storeFilesBuilder_ == null) {
             ensureStoreFilesIsMutable();
-            com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
                 values, storeFiles_);
             onChanged();
           } else {
@@ -2902,11 +2902,11 @@ public final class SnapshotProtos {
              getStoreFilesBuilderList() {
           return getStoreFilesFieldBuilder().getBuilderList();
         }
-        private com.google.protobuf.RepeatedFieldBuilderV3<
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
             org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
             getStoreFilesFieldBuilder() {
           if (storeFilesBuilder_ == null) {
-            storeFilesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+            storeFilesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
                 org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder>(
                     storeFiles_,
                     ((bitField0_ & 0x00000002) == 0x00000002),
@@ -2917,12 +2917,12 @@ public final class SnapshotProtos {
           return storeFilesBuilder_;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -2940,22 +2940,22 @@ public final class SnapshotProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public FamilyFiles parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new FamilyFiles(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -3061,7 +3061,7 @@ public final class SnapshotProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(1, version_);
@@ -3081,15 +3081,15 @@ public final class SnapshotProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt32Size(1, version_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getRegionInfo());
       }
       for (int i = 0; i < familyFiles_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, familyFiles_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -3150,61 +3150,61 @@ public final class SnapshotProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3222,7 +3222,7 @@ public final class SnapshotProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3230,15 +3230,15 @@ public final class SnapshotProtos {
      * Protobuf type {@code hbase.pb.SnapshotRegionManifest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotRegionManifest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3251,12 +3251,12 @@ public final class SnapshotProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getRegionInfoFieldBuilder();
           getFamilyFilesFieldBuilder();
@@ -3281,7 +3281,7 @@ public final class SnapshotProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
       }
@@ -3332,29 +3332,29 @@ public final class SnapshotProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest)other);
         } else {
@@ -3390,7 +3390,7 @@ public final class SnapshotProtos {
               familyFiles_ = other.familyFiles_;
               bitField0_ = (bitField0_ & ~0x00000004);
               familyFilesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getFamilyFilesFieldBuilder() : null;
             } else {
               familyFilesBuilder_.addAllMessages(other.familyFiles_);
@@ -3418,13 +3418,13 @@ public final class SnapshotProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3469,7 +3469,7 @@ public final class SnapshotProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
       /**
        * required .hbase.pb.RegionInfo region_info = 2;
@@ -3572,11 +3572,11 @@ public final class SnapshotProtos {
       /**
        * required .hbase.pb.RegionInfo region_info = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
           getRegionInfoFieldBuilder() {
         if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
                   getRegionInfo(),
                   getParentForChildren(),
@@ -3595,7 +3595,7 @@ public final class SnapshotProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> familyFilesBuilder_;
 
       /**
@@ -3727,7 +3727,7 @@ public final class SnapshotProtos {
           java.lang.Iterable values) {
         if (familyFilesBuilder_ == null) {
           ensureFamilyFilesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, familyFiles_);
           onChanged();
         } else {
@@ -3811,11 +3811,11 @@ public final class SnapshotProtos {
            getFamilyFilesBuilderList() {
         return getFamilyFilesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
           getFamilyFilesFieldBuilder() {
         if (familyFilesBuilder_ == null) {
-          familyFilesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          familyFilesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder>(
                   familyFiles_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -3826,12 +3826,12 @@ public final class SnapshotProtos {
         return familyFilesBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3849,22 +3849,22 @@ public final class SnapshotProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SnapshotRegionManifest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SnapshotRegionManifest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3876,7 +3876,7 @@ public final class SnapshotProtos {
 
   public interface SnapshotDataManifestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SnapshotDataManifest)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableSchema table_schema = 1;
@@ -3919,11 +3919,11 @@ public final class SnapshotProtos {
    * Protobuf type {@code hbase.pb.SnapshotDataManifest}
    */
   public  static final class SnapshotDataManifest extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SnapshotDataManifest)
       SnapshotDataManifestOrBuilder {
     // Use SnapshotDataManifest.newBuilder() to construct.
-    private SnapshotDataManifest(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SnapshotDataManifest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SnapshotDataManifest() {
@@ -3931,18 +3931,18 @@ public final class SnapshotProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SnapshotDataManifest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3982,10 +3982,10 @@ public final class SnapshotProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -3995,12 +3995,12 @@ public final class SnapshotProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4088,7 +4088,7 @@ public final class SnapshotProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableSchema());
@@ -4105,11 +4105,11 @@ public final class SnapshotProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableSchema());
       }
       for (int i = 0; i < regionManifests_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, regionManifests_.get(i));
       }
       size += unknownFields.getSerializedSize();
@@ -4161,61 +4161,61 @@ public final class SnapshotProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4233,7 +4233,7 @@ public final class SnapshotProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4241,15 +4241,15 @@ public final class SnapshotProtos {
      * Protobuf type {@code hbase.pb.SnapshotDataManifest}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SnapshotDataManifest)
         org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4262,12 +4262,12 @@ public final class SnapshotProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableSchemaFieldBuilder();
           getRegionManifestsFieldBuilder();
@@ -4290,7 +4290,7 @@ public final class SnapshotProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
       }
@@ -4337,29 +4337,29 @@ public final class SnapshotProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest)other);
         } else {
@@ -4392,7 +4392,7 @@ public final class SnapshotProtos {
               regionManifests_ = other.regionManifests_;
               bitField0_ = (bitField0_ & ~0x00000002);
               regionManifestsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionManifestsFieldBuilder() : null;
             } else {
               regionManifestsBuilder_.addAllMessages(other.regionManifests_);
@@ -4420,13 +4420,13 @@ public final class SnapshotProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4439,7 +4439,7 @@ public final class SnapshotProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
        * required .hbase.pb.TableSchema table_schema = 1;
@@ -4542,11 +4542,11 @@ public final class SnapshotProtos {
       /**
        * required .hbase.pb.TableSchema table_schema = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
           getTableSchemaFieldBuilder() {
         if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableSchemaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
                   getTableSchema(),
                   getParentForChildren(),
@@ -4565,7 +4565,7 @@ public final class SnapshotProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> regionManifestsBuilder_;
 
       /**
@@ -4697,7 +4697,7 @@ public final class SnapshotProtos {
           java.lang.Iterable values) {
         if (regionManifestsBuilder_ == null) {
           ensureRegionManifestsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, regionManifests_);
           onChanged();
         } else {
@@ -4781,11 +4781,11 @@ public final class SnapshotProtos {
            getRegionManifestsBuilderList() {
         return getRegionManifestsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
           getRegionManifestsFieldBuilder() {
         if (regionManifestsBuilder_ == null) {
-          regionManifestsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          regionManifestsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder>(
                   regionManifests_,
                   ((bitField0_ & 0x00000002) == 0x00000002),
@@ -4796,12 +4796,12 @@ public final class SnapshotProtos {
         return regionManifestsBuilder_;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4819,22 +4819,22 @@ public final class SnapshotProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SnapshotDataManifest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SnapshotDataManifest(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4844,37 +4844,37 @@ public final class SnapshotProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotFileInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotDataManifest_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -4898,48 +4898,48 @@ public final class SnapshotProtos {
       "che.hadoop.hbase.shaded.protobuf.generat" +
       "edB\016SnapshotProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_SnapshotFileInfo_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotFileInfo_descriptor,
         new java.lang.String[] { "Type", "Hfile", "WalServer", "WalName", });
     internal_static_hbase_pb_SnapshotRegionManifest_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotRegionManifest_descriptor,
         new java.lang.String[] { "Version", "RegionInfo", "FamilyFiles", });
     internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor =
       internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor,
         new java.lang.String[] { "Name", "Reference", "FileSize", });
     internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor =
       internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(1);
     internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor,
         new java.lang.String[] { "FamilyName", "StoreFiles", });
     internal_static_hbase_pb_SnapshotDataManifest_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SnapshotDataManifest_descriptor,
         new java.lang.String[] { "TableSchema", "RegionManifests", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
index 940a498..755bf65 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class TracingProtos {
   private TracingProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface RPCTInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RPCTInfo)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional int64 trace_id = 1;
@@ -48,11 +48,11 @@ public final class TracingProtos {
    * Protobuf type {@code hbase.pb.RPCTInfo}
    */
   public  static final class RPCTInfo extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RPCTInfo)
       RPCTInfoOrBuilder {
     // Use RPCTInfo.newBuilder() to construct.
-    private RPCTInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RPCTInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RPCTInfo() {
@@ -61,18 +61,18 @@ public final class TracingProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RPCTInfo(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -100,22 +100,22 @@ public final class TracingProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.internal_static_hbase_pb_RPCTInfo_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.internal_static_hbase_pb_RPCTInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -163,7 +163,7 @@ public final class TracingProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, traceId_);
@@ -180,11 +180,11 @@ public final class TracingProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, traceId_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(2, parentId_);
       }
       size += unknownFields.getSerializedSize();
@@ -227,12 +227,12 @@ public final class TracingProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasTraceId()) {
         hash = (37 * hash) + TRACE_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getTraceId());
       }
       if (hasParentId()) {
         hash = (37 * hash) + PARENT_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getParentId());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -241,61 +241,61 @@ public final class TracingProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -313,7 +313,7 @@ public final class TracingProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -329,15 +329,15 @@ public final class TracingProtos {
      * Protobuf type {@code hbase.pb.RPCTInfo}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RPCTInfo)
         org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfoOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.internal_static_hbase_pb_RPCTInfo_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.internal_static_hbase_pb_RPCTInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -350,12 +350,12 @@ public final class TracingProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -368,7 +368,7 @@ public final class TracingProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.internal_static_hbase_pb_RPCTInfo_descriptor;
       }
@@ -406,29 +406,29 @@ public final class TracingProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo)other);
         } else {
@@ -455,13 +455,13 @@ public final class TracingProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -537,12 +537,12 @@ public final class TracingProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -560,22 +560,22 @@ public final class TracingProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RPCTInfo parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RPCTInfo(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -585,17 +585,17 @@ public final class TracingProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RPCTInfo_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RPCTInfo_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -604,22 +604,22 @@ public final class TracingProtos {
       "g.apache.hadoop.hbase.shaded.protobuf.ge" +
       "neratedB\rTracingProtosH\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
         }, assigner);
     internal_static_hbase_pb_RPCTInfo_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_RPCTInfo_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RPCTInfo_descriptor,
         new java.lang.String[] { "TraceId", "ParentId", });
   }
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
index e032be7..db73e41 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
@@ -6,19 +6,19 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class WALProtos {
   private WALProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   /**
    * Protobuf enum {@code hbase.pb.ScopeType}
    */
   public enum ScopeType
-      implements com.google.protobuf.ProtocolMessageEnum {
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
      * REPLICATION_SCOPE_LOCAL = 0;
      */
@@ -68,27 +68,27 @@ public final class WALProtos {
       }
     }
 
-    public static com.google.protobuf.Internal.EnumLiteMap
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
         internalGetValueMap() {
       return internalValueMap;
     }
-    private static final com.google.protobuf.Internal.EnumLiteMap<
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
         ScopeType> internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap() {
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
             public ScopeType findValueByNumber(int number) {
               return ScopeType.forNumber(number);
             }
           };
 
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
         getValueDescriptor() {
       return getDescriptor().getValues().get(ordinal());
     }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptorForType() {
       return getDescriptor();
     }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.getDescriptor().getEnumTypes().get(0);
     }
@@ -96,7 +96,7 @@ public final class WALProtos {
     private static final ScopeType[] VALUES = values();
 
     public static ScopeType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
           "EnumValueDescriptor is not for this type.");
@@ -115,7 +115,7 @@ public final class WALProtos {
 
   public interface WALHeaderOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WALHeader)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool has_compression = 1;
@@ -133,7 +133,7 @@ public final class WALProtos {
     /**
      * optional bytes encryption_key = 2;
      */
-    com.google.protobuf.ByteString getEncryptionKey();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncryptionKey();
 
     /**
      * optional bool has_tag_compression = 3;
@@ -155,7 +155,7 @@ public final class WALProtos {
     /**
      * optional string writer_cls_name = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getWriterClsNameBytes();
 
     /**
@@ -169,41 +169,41 @@ public final class WALProtos {
     /**
      * optional string cell_codec_cls_name = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCellCodecClsNameBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.WALHeader}
    */
   public  static final class WALHeader extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WALHeader)
       WALHeaderOrBuilder {
     // Use WALHeader.newBuilder() to construct.
-    private WALHeader(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WALHeader(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WALHeader() {
       hasCompression_ = false;
-      encryptionKey_ = com.google.protobuf.ByteString.EMPTY;
+      encryptionKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       hasTagCompression_ = false;
       writerClsName_ = "";
       cellCodecClsName_ = "";
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WALHeader(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -235,35 +235,35 @@ public final class WALProtos {
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               writerClsName_ = bs;
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000010;
               cellCodecClsName_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALHeader_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALHeader_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -287,7 +287,7 @@ public final class WALProtos {
     }
 
     public static final int ENCRYPTION_KEY_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString encryptionKey_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encryptionKey_;
     /**
      * optional bytes encryption_key = 2;
      */
@@ -297,7 +297,7 @@ public final class WALProtos {
     /**
      * optional bytes encryption_key = 2;
      */
-    public com.google.protobuf.ByteString getEncryptionKey() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncryptionKey() {
       return encryptionKey_;
     }
 
@@ -332,8 +332,8 @@ public final class WALProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           writerClsName_ = s;
@@ -344,17 +344,17 @@ public final class WALProtos {
     /**
      * optional string writer_cls_name = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getWriterClsNameBytes() {
       java.lang.Object ref = writerClsName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         writerClsName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -374,8 +374,8 @@ public final class WALProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           cellCodecClsName_ = s;
@@ -386,17 +386,17 @@ public final class WALProtos {
     /**
      * optional string cell_codec_cls_name = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCellCodecClsNameBytes() {
       java.lang.Object ref = cellCodecClsName_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         cellCodecClsName_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -410,7 +410,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, hasCompression_);
@@ -422,10 +422,10 @@ public final class WALProtos {
         output.writeBool(3, hasTagCompression_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, writerClsName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, writerClsName_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, cellCodecClsName_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, cellCodecClsName_);
       }
       unknownFields.writeTo(output);
     }
@@ -436,22 +436,22 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, hasCompression_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, encryptionKey_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, hasTagCompression_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, writerClsName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, writerClsName_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, cellCodecClsName_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, cellCodecClsName_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -508,7 +508,7 @@ public final class WALProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasHasCompression()) {
         hash = (37 * hash) + HAS_COMPRESSION_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getHasCompression());
       }
       if (hasEncryptionKey()) {
@@ -517,7 +517,7 @@ public final class WALProtos {
       }
       if (hasHasTagCompression()) {
         hash = (37 * hash) + HAS_TAG_COMPRESSION_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getHasTagCompression());
       }
       if (hasWriterClsName()) {
@@ -534,61 +534,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -606,7 +606,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -614,15 +614,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.WALHeader}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WALHeader)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeaderOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALHeader_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALHeader_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -635,12 +635,12 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -648,7 +648,7 @@ public final class WALProtos {
         super.clear();
         hasCompression_ = false;
         bitField0_ = (bitField0_ & ~0x00000001);
-        encryptionKey_ = com.google.protobuf.ByteString.EMPTY;
+        encryptionKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         hasTagCompression_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -659,7 +659,7 @@ public final class WALProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALHeader_descriptor;
       }
@@ -709,29 +709,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader)other);
         } else {
@@ -771,13 +771,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -821,7 +821,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString encryptionKey_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encryptionKey_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * optional bytes encryption_key = 2;
        */
@@ -831,13 +831,13 @@ public final class WALProtos {
       /**
        * optional bytes encryption_key = 2;
        */
-      public com.google.protobuf.ByteString getEncryptionKey() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncryptionKey() {
         return encryptionKey_;
       }
       /**
        * optional bytes encryption_key = 2;
        */
-      public Builder setEncryptionKey(com.google.protobuf.ByteString value) {
+      public Builder setEncryptionKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -901,8 +901,8 @@ public final class WALProtos {
       public java.lang.String getWriterClsName() {
         java.lang.Object ref = writerClsName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             writerClsName_ = s;
@@ -915,17 +915,17 @@ public final class WALProtos {
       /**
        * optional string writer_cls_name = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getWriterClsNameBytes() {
         java.lang.Object ref = writerClsName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           writerClsName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -954,7 +954,7 @@ public final class WALProtos {
        * optional string writer_cls_name = 4;
        */
       public Builder setWriterClsNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -977,8 +977,8 @@ public final class WALProtos {
       public java.lang.String getCellCodecClsName() {
         java.lang.Object ref = cellCodecClsName_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             cellCodecClsName_ = s;
@@ -991,17 +991,17 @@ public final class WALProtos {
       /**
        * optional string cell_codec_cls_name = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getCellCodecClsNameBytes() {
         java.lang.Object ref = cellCodecClsName_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           cellCodecClsName_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -1030,7 +1030,7 @@ public final class WALProtos {
        * optional string cell_codec_cls_name = 5;
        */
       public Builder setCellCodecClsNameBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -1040,12 +1040,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1063,22 +1063,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WALHeader parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WALHeader(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1090,7 +1090,7 @@ public final class WALProtos {
 
   public interface WALKeyOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WALKey)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes encoded_region_name = 1;
@@ -1099,7 +1099,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 1;
      */
-    com.google.protobuf.ByteString getEncodedRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName();
 
     /**
      * required bytes table_name = 2;
@@ -1108,7 +1108,7 @@ public final class WALProtos {
     /**
      * required bytes table_name = 2;
      */
-    com.google.protobuf.ByteString getTableName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName();
 
     /**
      * required uint64 log_sequence_number = 3;
@@ -1280,16 +1280,16 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.WALKey}
    */
   public  static final class WALKey extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WALKey)
       WALKeyOrBuilder {
     // Use WALKey.newBuilder() to construct.
-    private WALKey(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WALKey(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WALKey() {
-      encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
-      tableName_ = com.google.protobuf.ByteString.EMPTY;
+      encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       logSequenceNumber_ = 0L;
       writeTime_ = 0L;
       scopes_ = java.util.Collections.emptyList();
@@ -1301,18 +1301,18 @@ public final class WALProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WALKey(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1401,10 +1401,10 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
@@ -1417,12 +1417,12 @@ public final class WALProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALKey_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALKey_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1431,7 +1431,7 @@ public final class WALProtos {
 
     private int bitField0_;
     public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString encodedRegionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_;
     /**
      * required bytes encoded_region_name = 1;
      */
@@ -1441,12 +1441,12 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 1;
      */
-    public com.google.protobuf.ByteString getEncodedRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
       return encodedRegionName_;
     }
 
     public static final int TABLE_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString tableName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_;
     /**
      * required bytes table_name = 2;
      */
@@ -1456,7 +1456,7 @@ public final class WALProtos {
     /**
      * required bytes table_name = 2;
      */
-    public com.google.protobuf.ByteString getTableName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
       return tableName_;
     }
 
@@ -1731,7 +1731,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, encodedRegionName_);
@@ -1775,47 +1775,47 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, encodedRegionName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, tableName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(3, logSequenceNumber_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, writeTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, getClusterId());
       }
       for (int i = 0; i < scopes_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, scopes_.get(i));
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(7, followingKvCount_);
       }
       for (int i = 0; i < clusterIds_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(8, clusterIds_.get(i));
       }
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(9, nonceGroup_);
       }
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(10, nonce_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(11, origSequenceNumber_);
       }
       size += unknownFields.getSerializedSize();
@@ -1905,12 +1905,12 @@ public final class WALProtos {
       }
       if (hasLogSequenceNumber()) {
         hash = (37 * hash) + LOG_SEQUENCE_NUMBER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLogSequenceNumber());
       }
       if (hasWriteTime()) {
         hash = (37 * hash) + WRITE_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getWriteTime());
       }
       if (hasClusterId()) {
@@ -1931,17 +1931,17 @@ public final class WALProtos {
       }
       if (hasNonceGroup()) {
         hash = (37 * hash) + NONCEGROUP_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonceGroup());
       }
       if (hasNonce()) {
         hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getNonce());
       }
       if (hasOrigSequenceNumber()) {
         hash = (37 * hash) + ORIG_SEQUENCE_NUMBER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getOrigSequenceNumber());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -1950,61 +1950,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2022,7 +2022,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2035,15 +2035,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.WALKey}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WALKey)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKeyOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALKey_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALKey_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2056,12 +2056,12 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getClusterIdFieldBuilder();
           getScopesFieldBuilder();
@@ -2070,9 +2070,9 @@ public final class WALProtos {
       }
       public Builder clear() {
         super.clear();
-        encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+        encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        tableName_ = com.google.protobuf.ByteString.EMPTY;
+        tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         logSequenceNumber_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -2107,7 +2107,7 @@ public final class WALProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALKey_descriptor;
       }
@@ -2195,29 +2195,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey)other);
         } else {
@@ -2262,7 +2262,7 @@ public final class WALProtos {
               scopes_ = other.scopes_;
               bitField0_ = (bitField0_ & ~0x00000020);
               scopesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getScopesFieldBuilder() : null;
             } else {
               scopesBuilder_.addAllMessages(other.scopes_);
@@ -2291,7 +2291,7 @@ public final class WALProtos {
               clusterIds_ = other.clusterIds_;
               bitField0_ = (bitField0_ & ~0x00000080);
               clusterIdsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getClusterIdsFieldBuilder() : null;
             } else {
               clusterIdsBuilder_.addAllMessages(other.clusterIds_);
@@ -2344,13 +2344,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2362,7 +2362,7 @@ public final class WALProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes encoded_region_name = 1;
        */
@@ -2372,13 +2372,13 @@ public final class WALProtos {
       /**
        * required bytes encoded_region_name = 1;
        */
-      public com.google.protobuf.ByteString getEncodedRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
         return encodedRegionName_;
       }
       /**
        * required bytes encoded_region_name = 1;
        */
-      public Builder setEncodedRegionName(com.google.protobuf.ByteString value) {
+      public Builder setEncodedRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2397,7 +2397,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString tableName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes table_name = 2;
        */
@@ -2407,13 +2407,13 @@ public final class WALProtos {
       /**
        * required bytes table_name = 2;
        */
-      public com.google.protobuf.ByteString getTableName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
         return tableName_;
       }
       /**
        * required bytes table_name = 2;
        */
-      public Builder setTableName(com.google.protobuf.ByteString value) {
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2497,7 +2497,7 @@ public final class WALProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID clusterId_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder> clusterIdBuilder_;
       /**
        * 
@@ -2663,11 +2663,11 @@ public final class WALProtos {
        *
        * optional .hbase.pb.UUID cluster_id = 5 [deprecated = true];
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder> 
           getClusterIdFieldBuilder() {
         if (clusterIdBuilder_ == null) {
-          clusterIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          clusterIdBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder>(
                   getClusterId(),
                   getParentForChildren(),
@@ -2686,7 +2686,7 @@ public final class WALProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScopeOrBuilder> scopesBuilder_;
 
       /**
@@ -2818,7 +2818,7 @@ public final class WALProtos {
           java.lang.Iterable values) {
         if (scopesBuilder_ == null) {
           ensureScopesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, scopes_);
           onChanged();
         } else {
@@ -2902,11 +2902,11 @@ public final class WALProtos {
            getScopesBuilderList() {
         return getScopesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScopeOrBuilder> 
           getScopesFieldBuilder() {
         if (scopesBuilder_ == null) {
-          scopesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          scopesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScopeOrBuilder>(
                   scopes_,
                   ((bitField0_ & 0x00000020) == 0x00000020),
@@ -2958,7 +2958,7 @@ public final class WALProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder> clusterIdsBuilder_;
 
       /**
@@ -3140,7 +3140,7 @@ public final class WALProtos {
           java.lang.Iterable values) {
         if (clusterIdsBuilder_ == null) {
           ensureClusterIdsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, clusterIds_);
           onChanged();
         } else {
@@ -3264,11 +3264,11 @@ public final class WALProtos {
            getClusterIdsBuilderList() {
         return getClusterIdsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder> 
           getClusterIdsFieldBuilder() {
         if (clusterIdsBuilder_ == null) {
-          clusterIdsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          clusterIdsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUIDOrBuilder>(
                   clusterIds_,
                   ((bitField0_ & 0x00000080) == 0x00000080),
@@ -3375,12 +3375,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3398,22 +3398,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WALKey parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WALKey(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3425,7 +3425,7 @@ public final class WALProtos {
 
   public interface FamilyScopeOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FamilyScope)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes family = 1;
@@ -3434,7 +3434,7 @@ public final class WALProtos {
     /**
      * required bytes family = 1;
      */
-    com.google.protobuf.ByteString getFamily();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily();
 
     /**
      * required .hbase.pb.ScopeType scope_type = 2;
@@ -3449,31 +3449,31 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.FamilyScope}
    */
   public  static final class FamilyScope extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FamilyScope)
       FamilyScopeOrBuilder {
     // Use FamilyScope.newBuilder() to construct.
-    private FamilyScope(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FamilyScope(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FamilyScope() {
-      family_ = com.google.protobuf.ByteString.EMPTY;
+      family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       scopeType_ = 0;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FamilyScope(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3507,22 +3507,22 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FamilyScope_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FamilyScope_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3531,7 +3531,7 @@ public final class WALProtos {
 
     private int bitField0_;
     public static final int FAMILY_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString family_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_;
     /**
      * required bytes family = 1;
      */
@@ -3541,7 +3541,7 @@ public final class WALProtos {
     /**
      * required bytes family = 1;
      */
-    public com.google.protobuf.ByteString getFamily() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
       return family_;
     }
 
@@ -3579,7 +3579,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, family_);
@@ -3596,11 +3596,11 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, family_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(2, scopeType_);
       }
       size += unknownFields.getSerializedSize();
@@ -3654,61 +3654,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3726,7 +3726,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3734,15 +3734,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.FamilyScope}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FamilyScope)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScopeOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FamilyScope_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FamilyScope_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3755,25 +3755,25 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        family_ = com.google.protobuf.ByteString.EMPTY;
+        family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         scopeType_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FamilyScope_descriptor;
       }
@@ -3811,29 +3811,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope)other);
         } else {
@@ -3866,13 +3866,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3884,7 +3884,7 @@ public final class WALProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString family_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family = 1;
        */
@@ -3894,13 +3894,13 @@ public final class WALProtos {
       /**
        * required bytes family = 1;
        */
-      public com.google.protobuf.ByteString getFamily() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamily() {
         return family_;
       }
       /**
        * required bytes family = 1;
        */
-      public Builder setFamily(com.google.protobuf.ByteString value) {
+      public Builder setFamily(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -3955,12 +3955,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3978,22 +3978,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FamilyScope parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FamilyScope(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4005,7 +4005,7 @@ public final class WALProtos {
 
   public interface CompactionDescriptorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CompactionDescriptor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -4022,7 +4022,7 @@ public final class WALProtos {
      *
      * required bytes table_name = 1;
      */
-    com.google.protobuf.ByteString getTableName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName();
 
     /**
      * required bytes encoded_region_name = 2;
@@ -4031,7 +4031,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 2;
      */
-    com.google.protobuf.ByteString getEncodedRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName();
 
     /**
      * required bytes family_name = 3;
@@ -4040,7 +4040,7 @@ public final class WALProtos {
     /**
      * required bytes family_name = 3;
      */
-    com.google.protobuf.ByteString getFamilyName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName();
 
     /**
      * 
@@ -4074,7 +4074,7 @@ public final class WALProtos {
      *
      * repeated string compaction_input = 4;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCompactionInputBytes(int index);
 
     /**
@@ -4093,7 +4093,7 @@ public final class WALProtos {
     /**
      * repeated string compaction_output = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCompactionOutputBytes(int index);
 
     /**
@@ -4119,7 +4119,7 @@ public final class WALProtos {
      *
      * required string store_home_dir = 6;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreHomeDirBytes();
 
     /**
@@ -4137,7 +4137,7 @@ public final class WALProtos {
      *
      * optional bytes region_name = 7;
      */
-    com.google.protobuf.ByteString getRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName();
   }
   /**
    * 
@@ -4151,36 +4151,36 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.CompactionDescriptor}
    */
   public  static final class CompactionDescriptor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.CompactionDescriptor)
       CompactionDescriptorOrBuilder {
     // Use CompactionDescriptor.newBuilder() to construct.
-    private CompactionDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private CompactionDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private CompactionDescriptor() {
-      tableName_ = com.google.protobuf.ByteString.EMPTY;
-      encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
-      familyName_ = com.google.protobuf.ByteString.EMPTY;
-      compactionInput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      compactionOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      compactionInput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
+      compactionOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       storeHomeDir_ = "";
-      regionName_ = com.google.protobuf.ByteString.EMPTY;
+      regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private CompactionDescriptor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4212,25 +4212,25 @@ public final class WALProtos {
               break;
             }
             case 34: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                compactionInput_ = new com.google.protobuf.LazyStringArrayList();
+                compactionInput_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                 mutable_bitField0_ |= 0x00000008;
               }
               compactionInput_.add(bs);
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                compactionOutput_ = new com.google.protobuf.LazyStringArrayList();
+                compactionOutput_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                 mutable_bitField0_ |= 0x00000010;
               }
               compactionOutput_.add(bs);
               break;
             }
             case 50: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000008;
               storeHomeDir_ = bs;
               break;
@@ -4242,10 +4242,10 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
@@ -4258,12 +4258,12 @@ public final class WALProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_CompactionDescriptor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_CompactionDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4272,7 +4272,7 @@ public final class WALProtos {
 
     private int bitField0_;
     public static final int TABLE_NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString tableName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_;
     /**
      * 
      * TODO: WALKey already stores these, might remove
@@ -4290,12 +4290,12 @@ public final class WALProtos {
      *
      * required bytes table_name = 1;
      */
-    public com.google.protobuf.ByteString getTableName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
       return tableName_;
     }
 
     public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString encodedRegionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_;
     /**
      * required bytes encoded_region_name = 2;
      */
@@ -4305,12 +4305,12 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 2;
      */
-    public com.google.protobuf.ByteString getEncodedRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
       return encodedRegionName_;
     }
 
     public static final int FAMILY_NAME_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString familyName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_;
     /**
      * required bytes family_name = 3;
      */
@@ -4320,12 +4320,12 @@ public final class WALProtos {
     /**
      * required bytes family_name = 3;
      */
-    public com.google.protobuf.ByteString getFamilyName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
       return familyName_;
     }
 
     public static final int COMPACTION_INPUT_FIELD_NUMBER = 4;
-    private com.google.protobuf.LazyStringList compactionInput_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList compactionInput_;
     /**
      * 
      * relative to store dir
@@ -4333,7 +4333,7 @@ public final class WALProtos {
      *
      * repeated string compaction_input = 4;
      */
-    public com.google.protobuf.ProtocolStringList
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
         getCompactionInputList() {
       return compactionInput_;
     }
@@ -4364,17 +4364,17 @@ public final class WALProtos {
      *
      * repeated string compaction_input = 4;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCompactionInputBytes(int index) {
       return compactionInput_.getByteString(index);
     }
 
     public static final int COMPACTION_OUTPUT_FIELD_NUMBER = 5;
-    private com.google.protobuf.LazyStringList compactionOutput_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList compactionOutput_;
     /**
      * repeated string compaction_output = 5;
      */
-    public com.google.protobuf.ProtocolStringList
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
         getCompactionOutputList() {
       return compactionOutput_;
     }
@@ -4393,7 +4393,7 @@ public final class WALProtos {
     /**
      * repeated string compaction_output = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getCompactionOutputBytes(int index) {
       return compactionOutput_.getByteString(index);
     }
@@ -4422,8 +4422,8 @@ public final class WALProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           storeHomeDir_ = s;
@@ -4438,22 +4438,22 @@ public final class WALProtos {
      *
      * required string store_home_dir = 6;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreHomeDirBytes() {
       java.lang.Object ref = storeHomeDir_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         storeHomeDir_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int REGION_NAME_FIELD_NUMBER = 7;
-    private com.google.protobuf.ByteString regionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_;
     /**
      * 
      * full region name
@@ -4471,7 +4471,7 @@ public final class WALProtos {
      *
      * optional bytes region_name = 7;
      */
-    public com.google.protobuf.ByteString getRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
       return regionName_;
     }
 
@@ -4501,7 +4501,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, tableName_);
@@ -4513,13 +4513,13 @@ public final class WALProtos {
         output.writeBytes(3, familyName_);
       }
       for (int i = 0; i < compactionInput_.size(); i++) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 4, compactionInput_.getRaw(i));
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, compactionInput_.getRaw(i));
       }
       for (int i = 0; i < compactionOutput_.size(); i++) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, compactionOutput_.getRaw(i));
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, compactionOutput_.getRaw(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 6, storeHomeDir_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 6, storeHomeDir_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeBytes(7, regionName_);
@@ -4533,15 +4533,15 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, tableName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, encodedRegionName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, familyName_);
       }
       {
@@ -4561,10 +4561,10 @@ public final class WALProtos {
         size += 1 * getCompactionOutputList().size();
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, storeHomeDir_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(6, storeHomeDir_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(7, regionName_);
       }
       size += unknownFields.getSerializedSize();
@@ -4658,61 +4658,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4730,7 +4730,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4746,15 +4746,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.CompactionDescriptor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.CompactionDescriptor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_CompactionDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_CompactionDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4767,35 +4767,35 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        tableName_ = com.google.protobuf.ByteString.EMPTY;
+        tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
-        encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+        encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        familyName_ = com.google.protobuf.ByteString.EMPTY;
+        familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
-        compactionInput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        compactionInput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
-        compactionOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        compactionOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000010);
         storeHomeDir_ = "";
         bitField0_ = (bitField0_ & ~0x00000020);
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_CompactionDescriptor_descriptor;
       }
@@ -4855,29 +4855,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor)other);
         } else {
@@ -4947,13 +4947,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4965,7 +4965,7 @@ public final class WALProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString tableName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        * TODO: WALKey already stores these, might remove
@@ -4983,7 +4983,7 @@ public final class WALProtos {
        *
        * required bytes table_name = 1;
        */
-      public com.google.protobuf.ByteString getTableName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
         return tableName_;
       }
       /**
@@ -4993,7 +4993,7 @@ public final class WALProtos {
        *
        * required bytes table_name = 1;
        */
-      public Builder setTableName(com.google.protobuf.ByteString value) {
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5016,7 +5016,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes encoded_region_name = 2;
        */
@@ -5026,13 +5026,13 @@ public final class WALProtos {
       /**
        * required bytes encoded_region_name = 2;
        */
-      public com.google.protobuf.ByteString getEncodedRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
         return encodedRegionName_;
       }
       /**
        * required bytes encoded_region_name = 2;
        */
-      public Builder setEncodedRegionName(com.google.protobuf.ByteString value) {
+      public Builder setEncodedRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5051,7 +5051,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family_name = 3;
        */
@@ -5061,13 +5061,13 @@ public final class WALProtos {
       /**
        * required bytes family_name = 3;
        */
-      public com.google.protobuf.ByteString getFamilyName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
         return familyName_;
       }
       /**
        * required bytes family_name = 3;
        */
-      public Builder setFamilyName(com.google.protobuf.ByteString value) {
+      public Builder setFamilyName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5086,10 +5086,10 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.LazyStringList compactionInput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList compactionInput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureCompactionInputIsMutable() {
         if (!((bitField0_ & 0x00000008) == 0x00000008)) {
-          compactionInput_ = new com.google.protobuf.LazyStringArrayList(compactionInput_);
+          compactionInput_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(compactionInput_);
           bitField0_ |= 0x00000008;
          }
       }
@@ -5100,7 +5100,7 @@ public final class WALProtos {
        *
        * repeated string compaction_input = 4;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getCompactionInputList() {
         return compactionInput_.getUnmodifiableView();
       }
@@ -5131,7 +5131,7 @@ public final class WALProtos {
        *
        * repeated string compaction_input = 4;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getCompactionInputBytes(int index) {
         return compactionInput_.getByteString(index);
       }
@@ -5179,7 +5179,7 @@ public final class WALProtos {
       public Builder addAllCompactionInput(
           java.lang.Iterable values) {
         ensureCompactionInputIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, compactionInput_);
         onChanged();
         return this;
@@ -5192,7 +5192,7 @@ public final class WALProtos {
        * repeated string compaction_input = 4;
        */
       public Builder clearCompactionInput() {
-        compactionInput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        compactionInput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
         onChanged();
         return this;
@@ -5205,7 +5205,7 @@ public final class WALProtos {
        * repeated string compaction_input = 4;
        */
       public Builder addCompactionInputBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5215,17 +5215,17 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.LazyStringList compactionOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList compactionOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureCompactionOutputIsMutable() {
         if (!((bitField0_ & 0x00000010) == 0x00000010)) {
-          compactionOutput_ = new com.google.protobuf.LazyStringArrayList(compactionOutput_);
+          compactionOutput_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(compactionOutput_);
           bitField0_ |= 0x00000010;
          }
       }
       /**
        * repeated string compaction_output = 5;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getCompactionOutputList() {
         return compactionOutput_.getUnmodifiableView();
       }
@@ -5244,7 +5244,7 @@ public final class WALProtos {
       /**
        * repeated string compaction_output = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getCompactionOutputBytes(int index) {
         return compactionOutput_.getByteString(index);
       }
@@ -5280,7 +5280,7 @@ public final class WALProtos {
       public Builder addAllCompactionOutput(
           java.lang.Iterable values) {
         ensureCompactionOutputIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, compactionOutput_);
         onChanged();
         return this;
@@ -5289,7 +5289,7 @@ public final class WALProtos {
        * repeated string compaction_output = 5;
        */
       public Builder clearCompactionOutput() {
-        compactionOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        compactionOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000010);
         onChanged();
         return this;
@@ -5298,7 +5298,7 @@ public final class WALProtos {
        * repeated string compaction_output = 5;
        */
       public Builder addCompactionOutputBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5329,8 +5329,8 @@ public final class WALProtos {
       public java.lang.String getStoreHomeDir() {
         java.lang.Object ref = storeHomeDir_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             storeHomeDir_ = s;
@@ -5347,17 +5347,17 @@ public final class WALProtos {
        *
        * required string store_home_dir = 6;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStoreHomeDirBytes() {
         java.lang.Object ref = storeHomeDir_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           storeHomeDir_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -5398,7 +5398,7 @@ public final class WALProtos {
        * required string store_home_dir = 6;
        */
       public Builder setStoreHomeDirBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5408,7 +5408,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        * full region name
@@ -5426,7 +5426,7 @@ public final class WALProtos {
        *
        * optional bytes region_name = 7;
        */
-      public com.google.protobuf.ByteString getRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
         return regionName_;
       }
       /**
@@ -5436,7 +5436,7 @@ public final class WALProtos {
        *
        * optional bytes region_name = 7;
        */
-      public Builder setRegionName(com.google.protobuf.ByteString value) {
+      public Builder setRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -5459,12 +5459,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -5482,22 +5482,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public CompactionDescriptor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new CompactionDescriptor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -5509,7 +5509,7 @@ public final class WALProtos {
 
   public interface FlushDescriptorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.FlushDescriptor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.FlushDescriptor.FlushAction action = 1;
@@ -5527,7 +5527,7 @@ public final class WALProtos {
     /**
      * required bytes table_name = 2;
      */
-    com.google.protobuf.ByteString getTableName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName();
 
     /**
      * required bytes encoded_region_name = 3;
@@ -5536,7 +5536,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 3;
      */
-    com.google.protobuf.ByteString getEncodedRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName();
 
     /**
      * optional uint64 flush_sequence_number = 4;
@@ -5586,7 +5586,7 @@ public final class WALProtos {
      *
      * optional bytes region_name = 6;
      */
-    com.google.protobuf.ByteString getRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName();
   }
   /**
    * 
@@ -5597,35 +5597,35 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.FlushDescriptor}
    */
   public  static final class FlushDescriptor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.FlushDescriptor)
       FlushDescriptorOrBuilder {
     // Use FlushDescriptor.newBuilder() to construct.
-    private FlushDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private FlushDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private FlushDescriptor() {
       action_ = 0;
-      tableName_ = com.google.protobuf.ByteString.EMPTY;
-      encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       flushSequenceNumber_ = 0L;
       storeFlushes_ = java.util.Collections.emptyList();
-      regionName_ = com.google.protobuf.ByteString.EMPTY;
+      regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private FlushDescriptor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5683,10 +5683,10 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
@@ -5696,12 +5696,12 @@ public final class WALProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5712,7 +5712,7 @@ public final class WALProtos {
      * Protobuf enum {@code hbase.pb.FlushDescriptor.FlushAction}
      */
     public enum FlushAction
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * START_FLUSH = 0;
        */
@@ -5779,27 +5779,27 @@ public final class WALProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           FlushAction> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public FlushAction findValueByNumber(int number) {
                 return FlushAction.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.getDescriptor().getEnumTypes().get(0);
       }
@@ -5807,7 +5807,7 @@ public final class WALProtos {
       private static final FlushAction[] VALUES = values();
 
       public static FlushAction valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -5826,7 +5826,7 @@ public final class WALProtos {
 
     public interface StoreFlushDescriptorOrBuilder extends
         // @@protoc_insertion_point(interface_extends:hbase.pb.FlushDescriptor.StoreFlushDescriptor)
-        com.google.protobuf.MessageOrBuilder {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
       /**
        * required bytes family_name = 1;
@@ -5835,7 +5835,7 @@ public final class WALProtos {
       /**
        * required bytes family_name = 1;
        */
-      com.google.protobuf.ByteString getFamilyName();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName();
 
       /**
        * 
@@ -5860,7 +5860,7 @@ public final class WALProtos {
        *
        * required string store_home_dir = 2;
        */
-      com.google.protobuf.ByteString
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStoreHomeDirBytes();
 
       /**
@@ -5895,39 +5895,39 @@ public final class WALProtos {
        *
        * repeated string flush_output = 3;
        */
-      com.google.protobuf.ByteString
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getFlushOutputBytes(int index);
     }
     /**
      * Protobuf type {@code hbase.pb.FlushDescriptor.StoreFlushDescriptor}
      */
     public  static final class StoreFlushDescriptor extends
-        com.google.protobuf.GeneratedMessageV3 implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
         // @@protoc_insertion_point(message_implements:hbase.pb.FlushDescriptor.StoreFlushDescriptor)
         StoreFlushDescriptorOrBuilder {
       // Use StoreFlushDescriptor.newBuilder() to construct.
-      private StoreFlushDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+      private StoreFlushDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
         super(builder);
       }
       private StoreFlushDescriptor() {
-        familyName_ = com.google.protobuf.ByteString.EMPTY;
+        familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         storeHomeDir_ = "";
-        flushOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        flushOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       }
 
       @java.lang.Override
-      public final com.google.protobuf.UnknownFieldSet
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
       getUnknownFields() {
         return this.unknownFields;
       }
       private StoreFlushDescriptor(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         this();
         int mutable_bitField0_ = 0;
-        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-            com.google.protobuf.UnknownFieldSet.newBuilder();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
         try {
           boolean done = false;
           while (!done) {
@@ -5949,15 +5949,15 @@ public final class WALProtos {
                 break;
               }
               case 18: {
-                com.google.protobuf.ByteString bs = input.readBytes();
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
                 bitField0_ |= 0x00000002;
                 storeHomeDir_ = bs;
                 break;
               }
               case 26: {
-                com.google.protobuf.ByteString bs = input.readBytes();
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
                 if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                  flushOutput_ = new com.google.protobuf.LazyStringArrayList();
+                  flushOutput_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                   mutable_bitField0_ |= 0x00000004;
                 }
                 flushOutput_.add(bs);
@@ -5965,10 +5965,10 @@ public final class WALProtos {
               }
             }
           }
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(this);
         } catch (java.io.IOException e) {
-          throw new com.google.protobuf.InvalidProtocolBufferException(
+          throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
               e).setUnfinishedMessage(this);
         } finally {
           if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -5978,12 +5978,12 @@ public final class WALProtos {
           makeExtensionsImmutable();
         }
       }
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5992,7 +5992,7 @@ public final class WALProtos {
 
       private int bitField0_;
       public static final int FAMILY_NAME_FIELD_NUMBER = 1;
-      private com.google.protobuf.ByteString familyName_;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_;
       /**
        * required bytes family_name = 1;
        */
@@ -6002,7 +6002,7 @@ public final class WALProtos {
       /**
        * required bytes family_name = 1;
        */
-      public com.google.protobuf.ByteString getFamilyName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
         return familyName_;
       }
 
@@ -6030,8 +6030,8 @@ public final class WALProtos {
         if (ref instanceof java.lang.String) {
           return (java.lang.String) ref;
         } else {
-          com.google.protobuf.ByteString bs = 
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             storeHomeDir_ = s;
@@ -6046,22 +6046,22 @@ public final class WALProtos {
        *
        * required string store_home_dir = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStoreHomeDirBytes() {
         java.lang.Object ref = storeHomeDir_;
         if (ref instanceof java.lang.String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           storeHomeDir_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
 
       public static final int FLUSH_OUTPUT_FIELD_NUMBER = 3;
-      private com.google.protobuf.LazyStringList flushOutput_;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList flushOutput_;
       /**
        * 
        * relative to store dir (if this is a COMMIT_FLUSH)
@@ -6069,7 +6069,7 @@ public final class WALProtos {
        *
        * repeated string flush_output = 3;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getFlushOutputList() {
         return flushOutput_;
       }
@@ -6100,7 +6100,7 @@ public final class WALProtos {
        *
        * repeated string flush_output = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getFlushOutputBytes(int index) {
         return flushOutput_.getByteString(index);
       }
@@ -6123,16 +6123,16 @@ public final class WALProtos {
         return true;
       }
 
-      public void writeTo(com.google.protobuf.CodedOutputStream output)
+      public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                           throws java.io.IOException {
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           output.writeBytes(1, familyName_);
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          com.google.protobuf.GeneratedMessageV3.writeString(output, 2, storeHomeDir_);
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, storeHomeDir_);
         }
         for (int i = 0; i < flushOutput_.size(); i++) {
-          com.google.protobuf.GeneratedMessageV3.writeString(output, 3, flushOutput_.getRaw(i));
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, flushOutput_.getRaw(i));
         }
         unknownFields.writeTo(output);
       }
@@ -6143,11 +6143,11 @@ public final class WALProtos {
 
         size = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          size += com.google.protobuf.CodedOutputStream
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSize(1, familyName_);
         }
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, storeHomeDir_);
+          size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, storeHomeDir_);
         }
         {
           int dataSize = 0;
@@ -6215,61 +6215,61 @@ public final class WALProtos {
       }
 
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(
-          com.google.protobuf.ByteString data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(
-          com.google.protobuf.ByteString data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(byte[] data)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(
           byte[] data,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
         return PARSER.parseFrom(data, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseDelimitedFrom(java.io.InputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseDelimitedFrom(
           java.io.InputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(
-          com.google.protobuf.CodedInputStream input)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input);
       }
       public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parseFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        return com.google.protobuf.GeneratedMessageV3
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
             .parseWithIOException(PARSER, input, extensionRegistry);
       }
 
@@ -6287,7 +6287,7 @@ public final class WALProtos {
 
       @java.lang.Override
       protected Builder newBuilderForType(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         Builder builder = new Builder(parent);
         return builder;
       }
@@ -6295,15 +6295,15 @@ public final class WALProtos {
        * Protobuf type {@code hbase.pb.FlushDescriptor.StoreFlushDescriptor}
        */
       public static final class Builder extends
-          com.google.protobuf.GeneratedMessageV3.Builder implements
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
           // @@protoc_insertion_point(builder_implements:hbase.pb.FlushDescriptor.StoreFlushDescriptor)
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptorOrBuilder {
-        public static final com.google.protobuf.Descriptors.Descriptor
+        public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_descriptor;
         }
 
-        protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
             internalGetFieldAccessorTable() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
@@ -6316,27 +6316,27 @@ public final class WALProtos {
         }
 
         private Builder(
-            com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
           super(parent);
           maybeForceBuilderInitialization();
         }
         private void maybeForceBuilderInitialization() {
-          if (com.google.protobuf.GeneratedMessageV3
+          if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                   .alwaysUseFieldBuilders) {
           }
         }
         public Builder clear() {
           super.clear();
-          familyName_ = com.google.protobuf.ByteString.EMPTY;
+          familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
           bitField0_ = (bitField0_ & ~0x00000001);
           storeHomeDir_ = "";
           bitField0_ = (bitField0_ & ~0x00000002);
-          flushOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+          flushOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
           bitField0_ = (bitField0_ & ~0x00000004);
           return this;
         }
 
-        public com.google.protobuf.Descriptors.Descriptor
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_descriptor;
         }
@@ -6379,29 +6379,29 @@ public final class WALProtos {
           return (Builder) super.clone();
         }
         public Builder setField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.setField(field, value);
         }
         public Builder clearField(
-            com.google.protobuf.Descriptors.FieldDescriptor field) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
           return (Builder) super.clearField(field);
         }
         public Builder clearOneof(
-            com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
           return (Builder) super.clearOneof(oneof);
         }
         public Builder setRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             int index, Object value) {
           return (Builder) super.setRepeatedField(field, index, value);
         }
         public Builder addRepeatedField(
-            com.google.protobuf.Descriptors.FieldDescriptor field,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
             Object value) {
           return (Builder) super.addRepeatedField(field, value);
         }
-        public Builder mergeFrom(com.google.protobuf.Message other) {
+        public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
           if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor) {
             return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor)other);
           } else {
@@ -6446,13 +6446,13 @@ public final class WALProtos {
         }
 
         public Builder mergeFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
             throws java.io.IOException {
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor parsedMessage = null;
           try {
             parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
             parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor) e.getUnfinishedMessage();
             throw e.unwrapIOException();
           } finally {
@@ -6464,7 +6464,7 @@ public final class WALProtos {
         }
         private int bitField0_;
 
-        private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         /**
          * required bytes family_name = 1;
          */
@@ -6474,13 +6474,13 @@ public final class WALProtos {
         /**
          * required bytes family_name = 1;
          */
-        public com.google.protobuf.ByteString getFamilyName() {
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
           return familyName_;
         }
         /**
          * required bytes family_name = 1;
          */
-        public Builder setFamilyName(com.google.protobuf.ByteString value) {
+        public Builder setFamilyName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -6520,8 +6520,8 @@ public final class WALProtos {
         public java.lang.String getStoreHomeDir() {
           java.lang.Object ref = storeHomeDir_;
           if (!(ref instanceof java.lang.String)) {
-            com.google.protobuf.ByteString bs =
-                (com.google.protobuf.ByteString) ref;
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+                (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
             java.lang.String s = bs.toStringUtf8();
             if (bs.isValidUtf8()) {
               storeHomeDir_ = s;
@@ -6538,17 +6538,17 @@ public final class WALProtos {
          *
          * required string store_home_dir = 2;
          */
-        public com.google.protobuf.ByteString
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
             getStoreHomeDirBytes() {
           java.lang.Object ref = storeHomeDir_;
           if (ref instanceof String) {
-            com.google.protobuf.ByteString b = 
-                com.google.protobuf.ByteString.copyFromUtf8(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                     (java.lang.String) ref);
             storeHomeDir_ = b;
             return b;
           } else {
-            return (com.google.protobuf.ByteString) ref;
+            return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           }
         }
         /**
@@ -6589,7 +6589,7 @@ public final class WALProtos {
          * required string store_home_dir = 2;
          */
         public Builder setStoreHomeDirBytes(
-            com.google.protobuf.ByteString value) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -6599,10 +6599,10 @@ public final class WALProtos {
           return this;
         }
 
-        private com.google.protobuf.LazyStringList flushOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList flushOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         private void ensureFlushOutputIsMutable() {
           if (!((bitField0_ & 0x00000004) == 0x00000004)) {
-            flushOutput_ = new com.google.protobuf.LazyStringArrayList(flushOutput_);
+            flushOutput_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(flushOutput_);
             bitField0_ |= 0x00000004;
            }
         }
@@ -6613,7 +6613,7 @@ public final class WALProtos {
          *
          * repeated string flush_output = 3;
          */
-        public com.google.protobuf.ProtocolStringList
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
             getFlushOutputList() {
           return flushOutput_.getUnmodifiableView();
         }
@@ -6644,7 +6644,7 @@ public final class WALProtos {
          *
          * repeated string flush_output = 3;
          */
-        public com.google.protobuf.ByteString
+        public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
             getFlushOutputBytes(int index) {
           return flushOutput_.getByteString(index);
         }
@@ -6692,7 +6692,7 @@ public final class WALProtos {
         public Builder addAllFlushOutput(
             java.lang.Iterable values) {
           ensureFlushOutputIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, flushOutput_);
           onChanged();
           return this;
@@ -6705,7 +6705,7 @@ public final class WALProtos {
          * repeated string flush_output = 3;
          */
         public Builder clearFlushOutput() {
-          flushOutput_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+          flushOutput_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
           bitField0_ = (bitField0_ & ~0x00000004);
           onChanged();
           return this;
@@ -6718,7 +6718,7 @@ public final class WALProtos {
          * repeated string flush_output = 3;
          */
         public Builder addFlushOutputBytes(
-            com.google.protobuf.ByteString value) {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
           if (value == null) {
     throw new NullPointerException();
   }
@@ -6728,12 +6728,12 @@ public final class WALProtos {
           return this;
         }
         public final Builder setUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.setUnknownFields(unknownFields);
         }
 
         public final Builder mergeUnknownFields(
-            final com.google.protobuf.UnknownFieldSet unknownFields) {
+            final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
           return super.mergeUnknownFields(unknownFields);
         }
 
@@ -6751,22 +6751,22 @@ public final class WALProtos {
         return DEFAULT_INSTANCE;
       }
 
-      @java.lang.Deprecated public static final com.google.protobuf.Parser
-          PARSER = new com.google.protobuf.AbstractParser() {
+      @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+          PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
         public StoreFlushDescriptor parsePartialFrom(
-            com.google.protobuf.CodedInputStream input,
-            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-            throws com.google.protobuf.InvalidProtocolBufferException {
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
             return new StoreFlushDescriptor(input, extensionRegistry);
         }
       };
 
-      public static com.google.protobuf.Parser parser() {
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
         return PARSER;
       }
 
       @java.lang.Override
-      public com.google.protobuf.Parser getParserForType() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
         return PARSER;
       }
 
@@ -6794,7 +6794,7 @@ public final class WALProtos {
     }
 
     public static final int TABLE_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString tableName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_;
     /**
      * required bytes table_name = 2;
      */
@@ -6804,12 +6804,12 @@ public final class WALProtos {
     /**
      * required bytes table_name = 2;
      */
-    public com.google.protobuf.ByteString getTableName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
       return tableName_;
     }
 
     public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString encodedRegionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_;
     /**
      * required bytes encoded_region_name = 3;
      */
@@ -6819,7 +6819,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 3;
      */
-    public com.google.protobuf.ByteString getEncodedRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
       return encodedRegionName_;
     }
 
@@ -6874,7 +6874,7 @@ public final class WALProtos {
     }
 
     public static final int REGION_NAME_FIELD_NUMBER = 6;
-    private com.google.protobuf.ByteString regionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_;
     /**
      * 
      * full region name
@@ -6892,7 +6892,7 @@ public final class WALProtos {
      *
      * optional bytes region_name = 6;
      */
-    public com.google.protobuf.ByteString getRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
       return regionName_;
     }
 
@@ -6924,7 +6924,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, action_);
@@ -6953,27 +6953,27 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, action_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, tableName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, encodedRegionName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, flushSequenceNumber_);
       }
       for (int i = 0; i < storeFlushes_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, storeFlushes_.get(i));
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, regionName_);
       }
       size += unknownFields.getSerializedSize();
@@ -7044,7 +7044,7 @@ public final class WALProtos {
       }
       if (hasFlushSequenceNumber()) {
         hash = (37 * hash) + FLUSH_SEQUENCE_NUMBER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getFlushSequenceNumber());
       }
       if (getStoreFlushesCount() > 0) {
@@ -7061,61 +7061,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7133,7 +7133,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7146,15 +7146,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.FlushDescriptor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.FlushDescriptor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7167,12 +7167,12 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getStoreFlushesFieldBuilder();
         }
@@ -7181,9 +7181,9 @@ public final class WALProtos {
         super.clear();
         action_ = 0;
         bitField0_ = (bitField0_ & ~0x00000001);
-        tableName_ = com.google.protobuf.ByteString.EMPTY;
+        tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+        encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         flushSequenceNumber_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
@@ -7193,12 +7193,12 @@ public final class WALProtos {
         } else {
           storeFlushesBuilder_.clear();
         }
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_FlushDescriptor_descriptor;
       }
@@ -7257,29 +7257,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor)other);
         } else {
@@ -7321,7 +7321,7 @@ public final class WALProtos {
               storeFlushes_ = other.storeFlushes_;
               bitField0_ = (bitField0_ & ~0x00000010);
               storeFlushesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStoreFlushesFieldBuilder() : null;
             } else {
               storeFlushesBuilder_.addAllMessages(other.storeFlushes_);
@@ -7355,13 +7355,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7409,7 +7409,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString tableName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes table_name = 2;
        */
@@ -7419,13 +7419,13 @@ public final class WALProtos {
       /**
        * required bytes table_name = 2;
        */
-      public com.google.protobuf.ByteString getTableName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
         return tableName_;
       }
       /**
        * required bytes table_name = 2;
        */
-      public Builder setTableName(com.google.protobuf.ByteString value) {
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -7444,7 +7444,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes encoded_region_name = 3;
        */
@@ -7454,13 +7454,13 @@ public final class WALProtos {
       /**
        * required bytes encoded_region_name = 3;
        */
-      public com.google.protobuf.ByteString getEncodedRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
         return encodedRegionName_;
       }
       /**
        * required bytes encoded_region_name = 3;
        */
-      public Builder setEncodedRegionName(com.google.protobuf.ByteString value) {
+      public Builder setEncodedRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -7520,7 +7520,7 @@ public final class WALProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptorOrBuilder> storeFlushesBuilder_;
 
       /**
@@ -7652,7 +7652,7 @@ public final class WALProtos {
           java.lang.Iterable values) {
         if (storeFlushesBuilder_ == null) {
           ensureStoreFlushesIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, storeFlushes_);
           onChanged();
         } else {
@@ -7736,11 +7736,11 @@ public final class WALProtos {
            getStoreFlushesBuilderList() {
         return getStoreFlushesFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptorOrBuilder> 
           getStoreFlushesFieldBuilder() {
         if (storeFlushesBuilder_ == null) {
-          storeFlushesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          storeFlushesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptorOrBuilder>(
                   storeFlushes_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -7751,7 +7751,7 @@ public final class WALProtos {
         return storeFlushesBuilder_;
       }
 
-      private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        * full region name
@@ -7769,7 +7769,7 @@ public final class WALProtos {
        *
        * optional bytes region_name = 6;
        */
-      public com.google.protobuf.ByteString getRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
         return regionName_;
       }
       /**
@@ -7779,7 +7779,7 @@ public final class WALProtos {
        *
        * optional bytes region_name = 6;
        */
-      public Builder setRegionName(com.google.protobuf.ByteString value) {
+      public Builder setRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -7802,12 +7802,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7825,22 +7825,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public FlushDescriptor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new FlushDescriptor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7852,7 +7852,7 @@ public final class WALProtos {
 
   public interface StoreDescriptorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.StoreDescriptor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required bytes family_name = 1;
@@ -7861,7 +7861,7 @@ public final class WALProtos {
     /**
      * required bytes family_name = 1;
      */
-    com.google.protobuf.ByteString getFamilyName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName();
 
     /**
      * 
@@ -7886,7 +7886,7 @@ public final class WALProtos {
      *
      * required string store_home_dir = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreHomeDirBytes();
 
     /**
@@ -7921,7 +7921,7 @@ public final class WALProtos {
      *
      * repeated string store_file = 3;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreFileBytes(int index);
 
     /**
@@ -7945,33 +7945,33 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.StoreDescriptor}
    */
   public  static final class StoreDescriptor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.StoreDescriptor)
       StoreDescriptorOrBuilder {
     // Use StoreDescriptor.newBuilder() to construct.
-    private StoreDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private StoreDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private StoreDescriptor() {
-      familyName_ = com.google.protobuf.ByteString.EMPTY;
+      familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       storeHomeDir_ = "";
-      storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       storeFileSizeBytes_ = 0L;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private StoreDescriptor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7993,15 +7993,15 @@ public final class WALProtos {
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               storeHomeDir_ = bs;
               break;
             }
             case 26: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                storeFile_ = new com.google.protobuf.LazyStringArrayList();
+                storeFile_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList();
                 mutable_bitField0_ |= 0x00000004;
               }
               storeFile_.add(bs);
@@ -8014,10 +8014,10 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -8027,12 +8027,12 @@ public final class WALProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_StoreDescriptor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_StoreDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8041,7 +8041,7 @@ public final class WALProtos {
 
     private int bitField0_;
     public static final int FAMILY_NAME_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString familyName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_;
     /**
      * required bytes family_name = 1;
      */
@@ -8051,7 +8051,7 @@ public final class WALProtos {
     /**
      * required bytes family_name = 1;
      */
-    public com.google.protobuf.ByteString getFamilyName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
       return familyName_;
     }
 
@@ -8079,8 +8079,8 @@ public final class WALProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           storeHomeDir_ = s;
@@ -8095,22 +8095,22 @@ public final class WALProtos {
      *
      * required string store_home_dir = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreHomeDirBytes() {
       java.lang.Object ref = storeHomeDir_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         storeHomeDir_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
     public static final int STORE_FILE_FIELD_NUMBER = 3;
-    private com.google.protobuf.LazyStringList storeFile_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList storeFile_;
     /**
      * 
      * relative to store dir
@@ -8118,7 +8118,7 @@ public final class WALProtos {
      *
      * repeated string store_file = 3;
      */
-    public com.google.protobuf.ProtocolStringList
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
         getStoreFileList() {
       return storeFile_;
     }
@@ -8149,7 +8149,7 @@ public final class WALProtos {
      *
      * repeated string store_file = 3;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStoreFileBytes(int index) {
       return storeFile_.getByteString(index);
     }
@@ -8195,16 +8195,16 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBytes(1, familyName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, storeHomeDir_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, storeHomeDir_);
       }
       for (int i = 0; i < storeFile_.size(); i++) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 3, storeFile_.getRaw(i));
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 3, storeFile_.getRaw(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeUInt64(4, storeFileSizeBytes_);
@@ -8218,11 +8218,11 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(1, familyName_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, storeHomeDir_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, storeHomeDir_);
       }
       {
         int dataSize = 0;
@@ -8233,7 +8233,7 @@ public final class WALProtos {
         size += 1 * getStoreFileList().size();
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, storeFileSizeBytes_);
       }
       size += unknownFields.getSerializedSize();
@@ -8295,7 +8295,7 @@ public final class WALProtos {
       }
       if (hasStoreFileSizeBytes()) {
         hash = (37 * hash) + STORE_FILE_SIZE_BYTES_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getStoreFileSizeBytes());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -8304,61 +8304,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8376,7 +8376,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8384,15 +8384,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.StoreDescriptor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.StoreDescriptor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_StoreDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_StoreDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8405,29 +8405,29 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
       public Builder clear() {
         super.clear();
-        familyName_ = com.google.protobuf.ByteString.EMPTY;
+        familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
         storeHomeDir_ = "";
         bitField0_ = (bitField0_ & ~0x00000002);
-        storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         storeFileSizeBytes_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_StoreDescriptor_descriptor;
       }
@@ -8474,29 +8474,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor)other);
         } else {
@@ -8544,13 +8544,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8562,7 +8562,7 @@ public final class WALProtos {
       }
       private int bitField0_;
 
-      private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString familyName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes family_name = 1;
        */
@@ -8572,13 +8572,13 @@ public final class WALProtos {
       /**
        * required bytes family_name = 1;
        */
-      public com.google.protobuf.ByteString getFamilyName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilyName() {
         return familyName_;
       }
       /**
        * required bytes family_name = 1;
        */
-      public Builder setFamilyName(com.google.protobuf.ByteString value) {
+      public Builder setFamilyName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8618,8 +8618,8 @@ public final class WALProtos {
       public java.lang.String getStoreHomeDir() {
         java.lang.Object ref = storeHomeDir_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             storeHomeDir_ = s;
@@ -8636,17 +8636,17 @@ public final class WALProtos {
        *
        * required string store_home_dir = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStoreHomeDirBytes() {
         java.lang.Object ref = storeHomeDir_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           storeHomeDir_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -8687,7 +8687,7 @@ public final class WALProtos {
        * required string store_home_dir = 2;
        */
       public Builder setStoreHomeDirBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8697,10 +8697,10 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.LazyStringList storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringList storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
       private void ensureStoreFileIsMutable() {
         if (!((bitField0_ & 0x00000004) == 0x00000004)) {
-          storeFile_ = new com.google.protobuf.LazyStringArrayList(storeFile_);
+          storeFile_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList(storeFile_);
           bitField0_ |= 0x00000004;
          }
       }
@@ -8711,7 +8711,7 @@ public final class WALProtos {
        *
        * repeated string store_file = 3;
        */
-      public com.google.protobuf.ProtocolStringList
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolStringList
           getStoreFileList() {
         return storeFile_.getUnmodifiableView();
       }
@@ -8742,7 +8742,7 @@ public final class WALProtos {
        *
        * repeated string store_file = 3;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStoreFileBytes(int index) {
         return storeFile_.getByteString(index);
       }
@@ -8790,7 +8790,7 @@ public final class WALProtos {
       public Builder addAllStoreFile(
           java.lang.Iterable values) {
         ensureStoreFileIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, storeFile_);
         onChanged();
         return this;
@@ -8803,7 +8803,7 @@ public final class WALProtos {
        * repeated string store_file = 3;
        */
       public Builder clearStoreFile() {
-        storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        storeFile_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         onChanged();
         return this;
@@ -8816,7 +8816,7 @@ public final class WALProtos {
        * repeated string store_file = 3;
        */
       public Builder addStoreFileBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -8874,12 +8874,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -8897,22 +8897,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public StoreDescriptor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new StoreDescriptor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8924,7 +8924,7 @@ public final class WALProtos {
 
   public interface BulkLoadDescriptorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.BulkLoadDescriptor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.TableName table_name = 1;
@@ -8946,7 +8946,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 2;
      */
-    com.google.protobuf.ByteString getEncodedRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName();
 
     /**
      * repeated .hbase.pb.StoreDescriptor stores = 3;
@@ -8990,32 +8990,32 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.BulkLoadDescriptor}
    */
   public  static final class BulkLoadDescriptor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.BulkLoadDescriptor)
       BulkLoadDescriptorOrBuilder {
     // Use BulkLoadDescriptor.newBuilder() to construct.
-    private BulkLoadDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private BulkLoadDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private BulkLoadDescriptor() {
-      encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       stores_ = java.util.Collections.emptyList();
       bulkloadSeqNum_ = 0L;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private BulkLoadDescriptor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9065,10 +9065,10 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -9078,12 +9078,12 @@ public final class WALProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_BulkLoadDescriptor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_BulkLoadDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9113,7 +9113,7 @@ public final class WALProtos {
     }
 
     public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString encodedRegionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_;
     /**
      * required bytes encoded_region_name = 2;
      */
@@ -9123,7 +9123,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 2;
      */
-    public com.google.protobuf.ByteString getEncodedRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
       return encodedRegionName_;
     }
 
@@ -9209,7 +9209,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -9232,19 +9232,19 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, encodedRegionName_);
       }
       for (int i = 0; i < stores_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, stores_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(4, bulkloadSeqNum_);
       }
       size += unknownFields.getSerializedSize();
@@ -9306,7 +9306,7 @@ public final class WALProtos {
       }
       if (hasBulkloadSeqNum()) {
         hash = (37 * hash) + BULKLOAD_SEQ_NUM_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getBulkloadSeqNum());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -9315,61 +9315,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9387,7 +9387,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9400,15 +9400,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.BulkLoadDescriptor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.BulkLoadDescriptor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_BulkLoadDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_BulkLoadDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9421,12 +9421,12 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getStoresFieldBuilder();
@@ -9440,7 +9440,7 @@ public final class WALProtos {
           tableNameBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+        encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
         if (storesBuilder_ == null) {
           stores_ = java.util.Collections.emptyList();
@@ -9453,7 +9453,7 @@ public final class WALProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_BulkLoadDescriptor_descriptor;
       }
@@ -9508,29 +9508,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor)other);
         } else {
@@ -9566,7 +9566,7 @@ public final class WALProtos {
               stores_ = other.stores_;
               bitField0_ = (bitField0_ & ~0x00000004);
               storesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStoresFieldBuilder() : null;
             } else {
               storesBuilder_.addAllMessages(other.stores_);
@@ -9603,13 +9603,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9622,7 +9622,7 @@ public final class WALProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * required .hbase.pb.TableName table_name = 1;
@@ -9725,11 +9725,11 @@ public final class WALProtos {
       /**
        * required .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -9739,7 +9739,7 @@ public final class WALProtos {
         return tableNameBuilder_;
       }
 
-      private com.google.protobuf.ByteString encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes encoded_region_name = 2;
        */
@@ -9749,13 +9749,13 @@ public final class WALProtos {
       /**
        * required bytes encoded_region_name = 2;
        */
-      public com.google.protobuf.ByteString getEncodedRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
         return encodedRegionName_;
       }
       /**
        * required bytes encoded_region_name = 2;
        */
-      public Builder setEncodedRegionName(com.google.protobuf.ByteString value) {
+      public Builder setEncodedRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -9783,7 +9783,7 @@ public final class WALProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder> storesBuilder_;
 
       /**
@@ -9915,7 +9915,7 @@ public final class WALProtos {
           java.lang.Iterable values) {
         if (storesBuilder_ == null) {
           ensureStoresIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, stores_);
           onChanged();
         } else {
@@ -9999,11 +9999,11 @@ public final class WALProtos {
            getStoresBuilderList() {
         return getStoresFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder> 
           getStoresFieldBuilder() {
         if (storesBuilder_ == null) {
-          storesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          storesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder>(
                   stores_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -10046,12 +10046,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -10069,22 +10069,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public BulkLoadDescriptor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new BulkLoadDescriptor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -10096,7 +10096,7 @@ public final class WALProtos {
 
   public interface RegionEventDescriptorOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionEventDescriptor)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.RegionEventDescriptor.EventType event_type = 1;
@@ -10114,7 +10114,7 @@ public final class WALProtos {
     /**
      * required bytes table_name = 2;
      */
-    com.google.protobuf.ByteString getTableName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName();
 
     /**
      * required bytes encoded_region_name = 3;
@@ -10123,7 +10123,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 3;
      */
-    com.google.protobuf.ByteString getEncodedRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName();
 
     /**
      * optional uint64 log_sequence_number = 4;
@@ -10198,7 +10198,7 @@ public final class WALProtos {
      *
      * optional bytes region_name = 7;
      */
-    com.google.protobuf.ByteString getRegionName();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName();
   }
   /**
    * 
@@ -10209,35 +10209,35 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.RegionEventDescriptor}
    */
   public  static final class RegionEventDescriptor extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.RegionEventDescriptor)
       RegionEventDescriptorOrBuilder {
     // Use RegionEventDescriptor.newBuilder() to construct.
-    private RegionEventDescriptor(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private RegionEventDescriptor(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private RegionEventDescriptor() {
       eventType_ = 0;
-      tableName_ = com.google.protobuf.ByteString.EMPTY;
-      encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
+      encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       logSequenceNumber_ = 0L;
       stores_ = java.util.Collections.emptyList();
-      regionName_ = com.google.protobuf.ByteString.EMPTY;
+      regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private RegionEventDescriptor(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -10308,10 +10308,10 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
@@ -10321,12 +10321,12 @@ public final class WALProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_RegionEventDescriptor_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_RegionEventDescriptor_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -10337,7 +10337,7 @@ public final class WALProtos {
      * Protobuf enum {@code hbase.pb.RegionEventDescriptor.EventType}
      */
     public enum EventType
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * REGION_OPEN = 0;
        */
@@ -10378,27 +10378,27 @@ public final class WALProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           EventType> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public EventType findValueByNumber(int number) {
                 return EventType.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.getDescriptor().getEnumTypes().get(0);
       }
@@ -10406,7 +10406,7 @@ public final class WALProtos {
       private static final EventType[] VALUES = values();
 
       public static EventType valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -10441,7 +10441,7 @@ public final class WALProtos {
     }
 
     public static final int TABLE_NAME_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString tableName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_;
     /**
      * required bytes table_name = 2;
      */
@@ -10451,12 +10451,12 @@ public final class WALProtos {
     /**
      * required bytes table_name = 2;
      */
-    public com.google.protobuf.ByteString getTableName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
       return tableName_;
     }
 
     public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 3;
-    private com.google.protobuf.ByteString encodedRegionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_;
     /**
      * required bytes encoded_region_name = 3;
      */
@@ -10466,7 +10466,7 @@ public final class WALProtos {
     /**
      * required bytes encoded_region_name = 3;
      */
-    public com.google.protobuf.ByteString getEncodedRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
       return encodedRegionName_;
     }
 
@@ -10554,7 +10554,7 @@ public final class WALProtos {
     }
 
     public static final int REGION_NAME_FIELD_NUMBER = 7;
-    private com.google.protobuf.ByteString regionName_;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_;
     /**
      * 
      * full region name
@@ -10572,7 +10572,7 @@ public final class WALProtos {
      *
      * optional bytes region_name = 7;
      */
-    public com.google.protobuf.ByteString getRegionName() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
       return regionName_;
     }
 
@@ -10610,7 +10610,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, eventType_);
@@ -10642,31 +10642,31 @@ public final class WALProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, eventType_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(2, tableName_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(3, encodedRegionName_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, logSequenceNumber_);
       }
       for (int i = 0; i < stores_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, stores_.get(i));
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(6, getServer());
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBytesSize(7, regionName_);
       }
       size += unknownFields.getSerializedSize();
@@ -10742,7 +10742,7 @@ public final class WALProtos {
       }
       if (hasLogSequenceNumber()) {
         hash = (37 * hash) + LOG_SEQUENCE_NUMBER_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getLogSequenceNumber());
       }
       if (getStoresCount() > 0) {
@@ -10763,61 +10763,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -10835,7 +10835,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -10848,15 +10848,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.RegionEventDescriptor}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.RegionEventDescriptor)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptorOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_RegionEventDescriptor_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_RegionEventDescriptor_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -10869,12 +10869,12 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getStoresFieldBuilder();
           getServerFieldBuilder();
@@ -10884,9 +10884,9 @@ public final class WALProtos {
         super.clear();
         eventType_ = 0;
         bitField0_ = (bitField0_ & ~0x00000001);
-        tableName_ = com.google.protobuf.ByteString.EMPTY;
+        tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000002);
-        encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+        encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000004);
         logSequenceNumber_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
@@ -10902,12 +10902,12 @@ public final class WALProtos {
           serverBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000020);
-        regionName_ = com.google.protobuf.ByteString.EMPTY;
+        regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_RegionEventDescriptor_descriptor;
       }
@@ -10974,29 +10974,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor)other);
         } else {
@@ -11038,7 +11038,7 @@ public final class WALProtos {
               stores_ = other.stores_;
               bitField0_ = (bitField0_ & ~0x00000010);
               storesBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getStoresFieldBuilder() : null;
             } else {
               storesBuilder_.addAllMessages(other.stores_);
@@ -11080,13 +11080,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -11134,7 +11134,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString tableName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString tableName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes table_name = 2;
        */
@@ -11144,13 +11144,13 @@ public final class WALProtos {
       /**
        * required bytes table_name = 2;
        */
-      public com.google.protobuf.ByteString getTableName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getTableName() {
         return tableName_;
       }
       /**
        * required bytes table_name = 2;
        */
-      public Builder setTableName(com.google.protobuf.ByteString value) {
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11169,7 +11169,7 @@ public final class WALProtos {
         return this;
       }
 
-      private com.google.protobuf.ByteString encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString encodedRegionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * required bytes encoded_region_name = 3;
        */
@@ -11179,13 +11179,13 @@ public final class WALProtos {
       /**
        * required bytes encoded_region_name = 3;
        */
-      public com.google.protobuf.ByteString getEncodedRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getEncodedRegionName() {
         return encodedRegionName_;
       }
       /**
        * required bytes encoded_region_name = 3;
        */
-      public Builder setEncodedRegionName(com.google.protobuf.ByteString value) {
+      public Builder setEncodedRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11245,7 +11245,7 @@ public final class WALProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder> storesBuilder_;
 
       /**
@@ -11377,7 +11377,7 @@ public final class WALProtos {
           java.lang.Iterable values) {
         if (storesBuilder_ == null) {
           ensureStoresIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, stores_);
           onChanged();
         } else {
@@ -11461,11 +11461,11 @@ public final class WALProtos {
            getStoresBuilderList() {
         return getStoresFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder> 
           getStoresFieldBuilder() {
         if (storesBuilder_ == null) {
-          storesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          storesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptorOrBuilder>(
                   stores_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -11477,7 +11477,7 @@ public final class WALProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
        * 
@@ -11616,11 +11616,11 @@ public final class WALProtos {
        *
        * optional .hbase.pb.ServerName server = 6;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerFieldBuilder() {
         if (serverBuilder_ == null) {
-          serverBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServer(),
                   getParentForChildren(),
@@ -11630,7 +11630,7 @@ public final class WALProtos {
         return serverBuilder_;
       }
 
-      private com.google.protobuf.ByteString regionName_ = com.google.protobuf.ByteString.EMPTY;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString regionName_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
       /**
        * 
        * full region name
@@ -11648,7 +11648,7 @@ public final class WALProtos {
        *
        * optional bytes region_name = 7;
        */
-      public com.google.protobuf.ByteString getRegionName() {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getRegionName() {
         return regionName_;
       }
       /**
@@ -11658,7 +11658,7 @@ public final class WALProtos {
        *
        * optional bytes region_name = 7;
        */
-      public Builder setRegionName(com.google.protobuf.ByteString value) {
+      public Builder setRegionName(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -11681,12 +11681,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -11704,22 +11704,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public RegionEventDescriptor parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new RegionEventDescriptor(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -11731,7 +11731,7 @@ public final class WALProtos {
 
   public interface WALTrailerOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.WALTrailer)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
   }
   /**
    * 
@@ -11745,28 +11745,28 @@ public final class WALProtos {
    * Protobuf type {@code hbase.pb.WALTrailer}
    */
   public  static final class WALTrailer extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.WALTrailer)
       WALTrailerOrBuilder {
     // Use WALTrailer.newBuilder() to construct.
-    private WALTrailer(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private WALTrailer(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private WALTrailer() {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private WALTrailer(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -11784,22 +11784,22 @@ public final class WALProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALTrailer_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALTrailer_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -11816,7 +11816,7 @@ public final class WALProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       unknownFields.writeTo(output);
     }
@@ -11860,61 +11860,61 @@ public final class WALProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -11932,7 +11932,7 @@ public final class WALProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -11948,15 +11948,15 @@ public final class WALProtos {
      * Protobuf type {@code hbase.pb.WALTrailer}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.WALTrailer)
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailerOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALTrailer_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALTrailer_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -11969,12 +11969,12 @@ public final class WALProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -11983,7 +11983,7 @@ public final class WALProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.internal_static_hbase_pb_WALTrailer_descriptor;
       }
@@ -12010,29 +12010,29 @@ public final class WALProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer)other);
         } else {
@@ -12053,13 +12053,13 @@ public final class WALProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -12070,12 +12070,12 @@ public final class WALProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -12093,22 +12093,22 @@ public final class WALProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public WALTrailer parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new WALTrailer(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -12118,62 +12118,62 @@ public final class WALProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WALHeader_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WALHeader_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WALKey_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WALKey_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FamilyScope_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FamilyScope_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CompactionDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CompactionDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FlushDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FlushDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_StoreDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_StoreDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_BulkLoadDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_BulkLoadDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionEventDescriptor_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionEventDescriptor_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_WALTrailer_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_WALTrailer_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -12228,77 +12228,77 @@ public final class WALProtos {
       "ase.shaded.protobuf.generatedB\tWALProtos" +
       "H\001\210\001\000\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_WALHeader_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_WALHeader_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WALHeader_descriptor,
         new java.lang.String[] { "HasCompression", "EncryptionKey", "HasTagCompression", "WriterClsName", "CellCodecClsName", });
     internal_static_hbase_pb_WALKey_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_WALKey_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WALKey_descriptor,
         new java.lang.String[] { "EncodedRegionName", "TableName", "LogSequenceNumber", "WriteTime", "ClusterId", "Scopes", "FollowingKvCount", "ClusterIds", "NonceGroup", "Nonce", "OrigSequenceNumber", });
     internal_static_hbase_pb_FamilyScope_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_FamilyScope_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FamilyScope_descriptor,
         new java.lang.String[] { "Family", "ScopeType", });
     internal_static_hbase_pb_CompactionDescriptor_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_CompactionDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_CompactionDescriptor_descriptor,
         new java.lang.String[] { "TableName", "EncodedRegionName", "FamilyName", "CompactionInput", "CompactionOutput", "StoreHomeDir", "RegionName", });
     internal_static_hbase_pb_FlushDescriptor_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_FlushDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FlushDescriptor_descriptor,
         new java.lang.String[] { "Action", "TableName", "EncodedRegionName", "FlushSequenceNumber", "StoreFlushes", "RegionName", });
     internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_descriptor =
       internal_static_hbase_pb_FlushDescriptor_descriptor.getNestedTypes().get(0);
     internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_FlushDescriptor_StoreFlushDescriptor_descriptor,
         new java.lang.String[] { "FamilyName", "StoreHomeDir", "FlushOutput", });
     internal_static_hbase_pb_StoreDescriptor_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_StoreDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_StoreDescriptor_descriptor,
         new java.lang.String[] { "FamilyName", "StoreHomeDir", "StoreFile", "StoreFileSizeBytes", });
     internal_static_hbase_pb_BulkLoadDescriptor_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_BulkLoadDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_BulkLoadDescriptor_descriptor,
         new java.lang.String[] { "TableName", "EncodedRegionName", "Stores", "BulkloadSeqNum", });
     internal_static_hbase_pb_RegionEventDescriptor_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_RegionEventDescriptor_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_RegionEventDescriptor_descriptor,
         new java.lang.String[] { "EventType", "TableName", "EncodedRegionName", "LogSequenceNumber", "Stores", "Server", "RegionName", });
     internal_static_hbase_pb_WALTrailer_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_WALTrailer_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_WALTrailer_descriptor,
         new java.lang.String[] { });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
index d7b5221..6baf845 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
@@ -6,17 +6,17 @@ package org.apache.hadoop.hbase.shaded.protobuf.generated;
 public final class ZooKeeperProtos {
   private ZooKeeperProtos() {}
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
   }
 
   public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
     registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
   public interface MetaRegionServerOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.MetaRegionServer)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -93,11 +93,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.MetaRegionServer}
    */
   public  static final class MetaRegionServer extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.MetaRegionServer)
       MetaRegionServerOrBuilder {
     // Use MetaRegionServer.newBuilder() to construct.
-    private MetaRegionServer(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private MetaRegionServer(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private MetaRegionServer() {
@@ -106,18 +106,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private MetaRegionServer(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -164,22 +164,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_MetaRegionServer_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_MetaRegionServer_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -292,7 +292,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getServer());
@@ -312,15 +312,15 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getServer());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, rpcVersion_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(3, state_);
       }
       size += unknownFields.getSerializedSize();
@@ -383,61 +383,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -455,7 +455,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -468,15 +468,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.MetaRegionServer}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.MetaRegionServer)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServerOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_MetaRegionServer_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_MetaRegionServer_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -489,12 +489,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerFieldBuilder();
         }
@@ -514,7 +514,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_MetaRegionServer_descriptor;
       }
@@ -560,29 +560,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer)other);
         } else {
@@ -618,13 +618,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -637,7 +637,7 @@ public final class ZooKeeperProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
        * 
@@ -785,11 +785,11 @@ public final class ZooKeeperProtos {
        *
        * required .hbase.pb.ServerName server = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerFieldBuilder() {
         if (serverBuilder_ == null) {
-          serverBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServer(),
                   getParentForChildren(),
@@ -907,12 +907,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -930,22 +930,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public MetaRegionServer parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new MetaRegionServer(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -957,7 +957,7 @@ public final class ZooKeeperProtos {
 
   public interface MasterOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.Master)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -1019,11 +1019,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.Master}
    */
   public  static final class Master extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.Master)
       MasterOrBuilder {
     // Use Master.newBuilder() to construct.
-    private Master(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private Master(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private Master() {
@@ -1032,18 +1032,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private Master(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1084,22 +1084,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Master_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Master_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1196,7 +1196,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getMaster());
@@ -1216,15 +1216,15 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getMaster());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(2, rpcVersion_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt32Size(3, infoPort_);
       }
       size += unknownFields.getSerializedSize();
@@ -1288,61 +1288,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -1360,7 +1360,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -1373,15 +1373,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.Master}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.Master)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MasterOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Master_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Master_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -1394,12 +1394,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getMasterFieldBuilder();
         }
@@ -1419,7 +1419,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_Master_descriptor;
       }
@@ -1465,29 +1465,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master)other);
         } else {
@@ -1523,13 +1523,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.Master) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -1542,7 +1542,7 @@ public final class ZooKeeperProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName master_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> masterBuilder_;
       /**
        * 
@@ -1681,11 +1681,11 @@ public final class ZooKeeperProtos {
        *
        * required .hbase.pb.ServerName master = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getMasterFieldBuilder() {
         if (masterBuilder_ == null) {
-          masterBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          masterBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getMaster(),
                   getParentForChildren(),
@@ -1775,12 +1775,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -1798,22 +1798,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public Master parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new Master(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -1825,7 +1825,7 @@ public final class ZooKeeperProtos {
 
   public interface ClusterUpOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ClusterUp)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -1853,7 +1853,7 @@ public final class ZooKeeperProtos {
      *
      * required string start_date = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStartDateBytes();
   }
   /**
@@ -1865,11 +1865,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.ClusterUp}
    */
   public  static final class ClusterUp extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ClusterUp)
       ClusterUpOrBuilder {
     // Use ClusterUp.newBuilder() to construct.
-    private ClusterUp(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ClusterUp(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ClusterUp() {
@@ -1877,18 +1877,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ClusterUp(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -1905,29 +1905,29 @@ public final class ZooKeeperProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               startDate_ = bs;
               break;
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ClusterUp_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ClusterUp_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -1961,8 +1961,8 @@ public final class ZooKeeperProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           startDate_ = s;
@@ -1978,17 +1978,17 @@ public final class ZooKeeperProtos {
      *
      * required string start_date = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStartDateBytes() {
       java.lang.Object ref = startDate_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         startDate_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -2006,10 +2006,10 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, startDate_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, startDate_);
       }
       unknownFields.writeTo(output);
     }
@@ -2020,7 +2020,7 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, startDate_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, startDate_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -2065,61 +2065,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -2137,7 +2137,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -2150,15 +2150,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.ClusterUp}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ClusterUp)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUpOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ClusterUp_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ClusterUp_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -2171,12 +2171,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -2187,7 +2187,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ClusterUp_descriptor;
       }
@@ -2221,29 +2221,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp)other);
         } else {
@@ -2272,13 +2272,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ClusterUp) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -2313,8 +2313,8 @@ public final class ZooKeeperProtos {
       public java.lang.String getStartDate() {
         java.lang.Object ref = startDate_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             startDate_ = s;
@@ -2332,17 +2332,17 @@ public final class ZooKeeperProtos {
        *
        * required string start_date = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getStartDateBytes() {
         java.lang.Object ref = startDate_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           startDate_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -2386,7 +2386,7 @@ public final class ZooKeeperProtos {
        * required string start_date = 1;
        */
       public Builder setStartDateBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -2396,12 +2396,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -2419,22 +2419,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ClusterUp parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ClusterUp(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -2446,7 +2446,7 @@ public final class ZooKeeperProtos {
 
   public interface SplitLogTaskOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SplitLogTask)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.SplitLogTask.State state = 1;
@@ -2489,11 +2489,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.SplitLogTask}
    */
   public  static final class SplitLogTask extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SplitLogTask)
       SplitLogTaskOrBuilder {
     // Use SplitLogTask.newBuilder() to construct.
-    private SplitLogTask(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SplitLogTask(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SplitLogTask() {
@@ -2502,18 +2502,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SplitLogTask(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -2566,22 +2566,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitLogTask_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitLogTask_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -2592,7 +2592,7 @@ public final class ZooKeeperProtos {
      * Protobuf enum {@code hbase.pb.SplitLogTask.State}
      */
     public enum State
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * UNASSIGNED = 0;
        */
@@ -2660,27 +2660,27 @@ public final class ZooKeeperProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           State> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public State findValueByNumber(int number) {
                 return State.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.getDescriptor().getEnumTypes().get(0);
       }
@@ -2688,7 +2688,7 @@ public final class ZooKeeperProtos {
       private static final State[] VALUES = values();
 
       public static State valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -2709,7 +2709,7 @@ public final class ZooKeeperProtos {
      * Protobuf enum {@code hbase.pb.SplitLogTask.RecoveryMode}
      */
     public enum RecoveryMode
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * UNKNOWN = 0;
        */
@@ -2759,27 +2759,27 @@ public final class ZooKeeperProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           RecoveryMode> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public RecoveryMode findValueByNumber(int number) {
                 return RecoveryMode.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.getDescriptor().getEnumTypes().get(1);
       }
@@ -2787,7 +2787,7 @@ public final class ZooKeeperProtos {
       private static final RecoveryMode[] VALUES = values();
 
       public static RecoveryMode valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -2880,7 +2880,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, state_);
@@ -2900,15 +2900,15 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, state_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getServerName());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(3, mode_);
       }
       size += unknownFields.getSerializedSize();
@@ -2970,61 +2970,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3042,7 +3042,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3056,15 +3056,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.SplitLogTask}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SplitLogTask)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTaskOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitLogTask_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitLogTask_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3077,12 +3077,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getServerNameFieldBuilder();
         }
@@ -3102,7 +3102,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SplitLogTask_descriptor;
       }
@@ -3148,29 +3148,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask)other);
         } else {
@@ -3209,13 +3209,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -3264,7 +3264,7 @@ public final class ZooKeeperProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_;
       /**
        * required .hbase.pb.ServerName server_name = 2;
@@ -3367,11 +3367,11 @@ public final class ZooKeeperProtos {
       /**
        * required .hbase.pb.ServerName server_name = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getServerNameFieldBuilder() {
         if (serverNameBuilder_ == null) {
-          serverNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          serverNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getServerName(),
                   getParentForChildren(),
@@ -3417,12 +3417,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -3440,22 +3440,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SplitLogTask parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SplitLogTask(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -3467,7 +3467,7 @@ public final class ZooKeeperProtos {
 
   public interface DeprecatedTableStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.DeprecatedTableState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -3500,11 +3500,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.DeprecatedTableState}
    */
   public  static final class DeprecatedTableState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.DeprecatedTableState)
       DeprecatedTableStateOrBuilder {
     // Use DeprecatedTableState.newBuilder() to construct.
-    private DeprecatedTableState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private DeprecatedTableState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private DeprecatedTableState() {
@@ -3512,18 +3512,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private DeprecatedTableState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -3552,22 +3552,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -3582,7 +3582,7 @@ public final class ZooKeeperProtos {
      * Protobuf enum {@code hbase.pb.DeprecatedTableState.State}
      */
     public enum State
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * ENABLED = 0;
        */
@@ -3641,27 +3641,27 @@ public final class ZooKeeperProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           State> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public State findValueByNumber(int number) {
                 return State.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState.getDescriptor().getEnumTypes().get(0);
       }
@@ -3669,7 +3669,7 @@ public final class ZooKeeperProtos {
       private static final State[] VALUES = values();
 
       public static State valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -3729,7 +3729,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, state_);
@@ -3743,7 +3743,7 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, state_);
       }
       size += unknownFields.getSerializedSize();
@@ -3788,61 +3788,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -3860,7 +3860,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -3874,15 +3874,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.DeprecatedTableState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.DeprecatedTableState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -3895,12 +3895,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -3911,7 +3911,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_DeprecatedTableState_descriptor;
       }
@@ -3945,29 +3945,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState)other);
         } else {
@@ -3994,13 +3994,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.DeprecatedTableState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4072,12 +4072,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4095,22 +4095,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public DeprecatedTableState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new DeprecatedTableState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4122,7 +4122,7 @@ public final class ZooKeeperProtos {
 
   public interface TableCFOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TableCF)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.TableName table_name = 1;
@@ -4140,7 +4140,7 @@ public final class ZooKeeperProtos {
     /**
      * repeated bytes families = 2;
      */
-    java.util.List getFamiliesList();
+    java.util.List getFamiliesList();
     /**
      * repeated bytes families = 2;
      */
@@ -4148,17 +4148,17 @@ public final class ZooKeeperProtos {
     /**
      * repeated bytes families = 2;
      */
-    com.google.protobuf.ByteString getFamilies(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index);
   }
   /**
    * Protobuf type {@code hbase.pb.TableCF}
    */
   public  static final class TableCF extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TableCF)
       TableCFOrBuilder {
     // Use TableCF.newBuilder() to construct.
-    private TableCF(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TableCF(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TableCF() {
@@ -4166,18 +4166,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TableCF(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -4208,7 +4208,7 @@ public final class ZooKeeperProtos {
             }
             case 18: {
               if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                families_ = new java.util.ArrayList();
+                families_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000002;
               }
               families_.add(input.readBytes());
@@ -4216,10 +4216,10 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
@@ -4229,12 +4229,12 @@ public final class ZooKeeperProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -4264,11 +4264,11 @@ public final class ZooKeeperProtos {
     }
 
     public static final int FAMILIES_FIELD_NUMBER = 2;
-    private java.util.List families_;
+    private java.util.List families_;
     /**
      * repeated bytes families = 2;
      */
-    public java.util.List
+    public java.util.List
         getFamiliesList() {
       return families_;
     }
@@ -4281,7 +4281,7 @@ public final class ZooKeeperProtos {
     /**
      * repeated bytes families = 2;
      */
-    public com.google.protobuf.ByteString getFamilies(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
       return families_.get(index);
     }
 
@@ -4301,7 +4301,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -4318,13 +4318,13 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       {
         int dataSize = 0;
         for (int i = 0; i < families_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(families_.get(i));
         }
         size += dataSize;
@@ -4379,61 +4379,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -4451,7 +4451,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -4459,15 +4459,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.TableCF}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TableCF)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -4480,12 +4480,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
         }
@@ -4503,7 +4503,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
       }
@@ -4546,29 +4546,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF)other);
         } else {
@@ -4607,13 +4607,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -4626,7 +4626,7 @@ public final class ZooKeeperProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * optional .hbase.pb.TableName table_name = 1;
@@ -4729,11 +4729,11 @@ public final class ZooKeeperProtos {
       /**
        * optional .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -4743,17 +4743,17 @@ public final class ZooKeeperProtos {
         return tableNameBuilder_;
       }
 
-      private java.util.List families_ = java.util.Collections.emptyList();
+      private java.util.List families_ = java.util.Collections.emptyList();
       private void ensureFamiliesIsMutable() {
         if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = new java.util.ArrayList(families_);
+          families_ = new java.util.ArrayList(families_);
           bitField0_ |= 0x00000002;
          }
       }
       /**
        * repeated bytes families = 2;
        */
-      public java.util.List
+      public java.util.List
           getFamiliesList() {
         return java.util.Collections.unmodifiableList(families_);
       }
@@ -4766,14 +4766,14 @@ public final class ZooKeeperProtos {
       /**
        * repeated bytes families = 2;
        */
-      public com.google.protobuf.ByteString getFamilies(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
         return families_.get(index);
       }
       /**
        * repeated bytes families = 2;
        */
       public Builder setFamilies(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4785,7 +4785,7 @@ public final class ZooKeeperProtos {
       /**
        * repeated bytes families = 2;
        */
-      public Builder addFamilies(com.google.protobuf.ByteString value) {
+      public Builder addFamilies(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -4798,9 +4798,9 @@ public final class ZooKeeperProtos {
        * repeated bytes families = 2;
        */
       public Builder addAllFamilies(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureFamiliesIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, families_);
         onChanged();
         return this;
@@ -4815,12 +4815,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -4838,22 +4838,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TableCF parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TableCF(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -4865,7 +4865,7 @@ public final class ZooKeeperProtos {
 
   public interface ReplicationPeerOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationPeer)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * 
@@ -4893,7 +4893,7 @@ public final class ZooKeeperProtos {
      *
      * required string clusterkey = 1;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClusterkeyBytes();
 
     /**
@@ -4907,7 +4907,7 @@ public final class ZooKeeperProtos {
     /**
      * optional string replicationEndpointImpl = 2;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getReplicationEndpointImplBytes();
 
     /**
@@ -4985,7 +4985,7 @@ public final class ZooKeeperProtos {
     /**
      * repeated bytes namespaces = 6;
      */
-    java.util.List getNamespacesList();
+    java.util.List getNamespacesList();
     /**
      * repeated bytes namespaces = 6;
      */
@@ -4993,7 +4993,7 @@ public final class ZooKeeperProtos {
     /**
      * repeated bytes namespaces = 6;
      */
-    com.google.protobuf.ByteString getNamespaces(int index);
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index);
   }
   /**
    * 
@@ -5004,11 +5004,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.ReplicationPeer}
    */
   public  static final class ReplicationPeer extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationPeer)
       ReplicationPeerOrBuilder {
     // Use ReplicationPeer.newBuilder() to construct.
-    private ReplicationPeer(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicationPeer(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicationPeer() {
@@ -5021,18 +5021,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicationPeer(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -5049,13 +5049,13 @@ public final class ZooKeeperProtos {
               break;
             }
             case 10: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
               clusterkey_ = bs;
               break;
             }
             case 18: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000002;
               replicationEndpointImpl_ = bs;
               break;
@@ -5089,7 +5089,7 @@ public final class ZooKeeperProtos {
             }
             case 50: {
               if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                namespaces_ = new java.util.ArrayList();
+                namespaces_ = new java.util.ArrayList();
                 mutable_bitField0_ |= 0x00000020;
               }
               namespaces_.add(input.readBytes());
@@ -5097,10 +5097,10 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
@@ -5119,12 +5119,12 @@ public final class ZooKeeperProtos {
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -5158,8 +5158,8 @@ public final class ZooKeeperProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           clusterkey_ = s;
@@ -5175,17 +5175,17 @@ public final class ZooKeeperProtos {
      *
      * required string clusterkey = 1;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getClusterkeyBytes() {
       java.lang.Object ref = clusterkey_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         clusterkey_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -5205,8 +5205,8 @@ public final class ZooKeeperProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           replicationEndpointImpl_ = s;
@@ -5217,17 +5217,17 @@ public final class ZooKeeperProtos {
     /**
      * optional string replicationEndpointImpl = 2;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getReplicationEndpointImplBytes() {
       java.lang.Object ref = replicationEndpointImpl_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         replicationEndpointImpl_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -5337,11 +5337,11 @@ public final class ZooKeeperProtos {
     }
 
     public static final int NAMESPACES_FIELD_NUMBER = 6;
-    private java.util.List namespaces_;
+    private java.util.List namespaces_;
     /**
      * repeated bytes namespaces = 6;
      */
-    public java.util.List
+    public java.util.List
         getNamespacesList() {
       return namespaces_;
     }
@@ -5354,7 +5354,7 @@ public final class ZooKeeperProtos {
     /**
      * repeated bytes namespaces = 6;
      */
-    public com.google.protobuf.ByteString getNamespaces(int index) {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index) {
       return namespaces_.get(index);
     }
 
@@ -5390,13 +5390,13 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterkey_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterkey_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 2, replicationEndpointImpl_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, replicationEndpointImpl_);
       }
       for (int i = 0; i < data_.size(); i++) {
         output.writeMessage(3, data_.get(i));
@@ -5419,27 +5419,27 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterkey_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterkey_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationEndpointImpl_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationEndpointImpl_);
       }
       for (int i = 0; i < data_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, data_.get(i));
       }
       for (int i = 0; i < configuration_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(4, configuration_.get(i));
       }
       for (int i = 0; i < tableCfs_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(5, tableCfs_.get(i));
       }
       {
         int dataSize = 0;
         for (int i = 0; i < namespaces_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
             .computeBytesSizeNoTag(namespaces_.get(i));
         }
         size += dataSize;
@@ -5521,61 +5521,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -5593,7 +5593,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -5606,15 +5606,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.ReplicationPeer}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationPeer)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeerOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -5627,12 +5627,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getDataFieldBuilder();
           getConfigurationFieldBuilder();
@@ -5668,7 +5668,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
       }
@@ -5738,29 +5738,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer)other);
         } else {
@@ -5800,7 +5800,7 @@ public final class ZooKeeperProtos {
               data_ = other.data_;
               bitField0_ = (bitField0_ & ~0x00000004);
               dataBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getDataFieldBuilder() : null;
             } else {
               dataBuilder_.addAllMessages(other.data_);
@@ -5826,7 +5826,7 @@ public final class ZooKeeperProtos {
               configuration_ = other.configuration_;
               bitField0_ = (bitField0_ & ~0x00000008);
               configurationBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getConfigurationFieldBuilder() : null;
             } else {
               configurationBuilder_.addAllMessages(other.configuration_);
@@ -5852,7 +5852,7 @@ public final class ZooKeeperProtos {
               tableCfs_ = other.tableCfs_;
               bitField0_ = (bitField0_ & ~0x00000010);
               tableCfsBuilder_ = 
-                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableCfsFieldBuilder() : null;
             } else {
               tableCfsBuilder_.addAllMessages(other.tableCfs_);
@@ -5897,13 +5897,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -5938,8 +5938,8 @@ public final class ZooKeeperProtos {
       public java.lang.String getClusterkey() {
         java.lang.Object ref = clusterkey_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             clusterkey_ = s;
@@ -5957,17 +5957,17 @@ public final class ZooKeeperProtos {
        *
        * required string clusterkey = 1;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getClusterkeyBytes() {
         java.lang.Object ref = clusterkey_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           clusterkey_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -6011,7 +6011,7 @@ public final class ZooKeeperProtos {
        * required string clusterkey = 1;
        */
       public Builder setClusterkeyBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -6034,8 +6034,8 @@ public final class ZooKeeperProtos {
       public java.lang.String getReplicationEndpointImpl() {
         java.lang.Object ref = replicationEndpointImpl_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             replicationEndpointImpl_ = s;
@@ -6048,17 +6048,17 @@ public final class ZooKeeperProtos {
       /**
        * optional string replicationEndpointImpl = 2;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getReplicationEndpointImplBytes() {
         java.lang.Object ref = replicationEndpointImpl_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           replicationEndpointImpl_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -6087,7 +6087,7 @@ public final class ZooKeeperProtos {
        * optional string replicationEndpointImpl = 2;
        */
       public Builder setReplicationEndpointImplBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -6106,7 +6106,7 @@ public final class ZooKeeperProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> dataBuilder_;
 
       /**
@@ -6238,7 +6238,7 @@ public final class ZooKeeperProtos {
           java.lang.Iterable values) {
         if (dataBuilder_ == null) {
           ensureDataIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, data_);
           onChanged();
         } else {
@@ -6322,11 +6322,11 @@ public final class ZooKeeperProtos {
            getDataBuilderList() {
         return getDataFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
           getDataFieldBuilder() {
         if (dataBuilder_ == null) {
-          dataBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          dataBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>(
                   data_,
                   ((bitField0_ & 0x00000004) == 0x00000004),
@@ -6346,7 +6346,7 @@ public final class ZooKeeperProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> configurationBuilder_;
 
       /**
@@ -6478,7 +6478,7 @@ public final class ZooKeeperProtos {
           java.lang.Iterable values) {
         if (configurationBuilder_ == null) {
           ensureConfigurationIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, configuration_);
           onChanged();
         } else {
@@ -6562,11 +6562,11 @@ public final class ZooKeeperProtos {
            getConfigurationBuilderList() {
         return getConfigurationFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
           getConfigurationFieldBuilder() {
         if (configurationBuilder_ == null) {
-          configurationBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          configurationBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder>(
                   configuration_,
                   ((bitField0_ & 0x00000008) == 0x00000008),
@@ -6586,7 +6586,7 @@ public final class ZooKeeperProtos {
          }
       }
 
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> tableCfsBuilder_;
 
       /**
@@ -6718,7 +6718,7 @@ public final class ZooKeeperProtos {
           java.lang.Iterable values) {
         if (tableCfsBuilder_ == null) {
           ensureTableCfsIsMutable();
-          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
               values, tableCfs_);
           onChanged();
         } else {
@@ -6802,11 +6802,11 @@ public final class ZooKeeperProtos {
            getTableCfsBuilderList() {
         return getTableCfsFieldBuilder().getBuilderList();
       }
-      private com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
           getTableCfsFieldBuilder() {
         if (tableCfsBuilder_ == null) {
-          tableCfsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+          tableCfsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder>(
                   tableCfs_,
                   ((bitField0_ & 0x00000010) == 0x00000010),
@@ -6817,17 +6817,17 @@ public final class ZooKeeperProtos {
         return tableCfsBuilder_;
       }
 
-      private java.util.List namespaces_ = java.util.Collections.emptyList();
+      private java.util.List namespaces_ = java.util.Collections.emptyList();
       private void ensureNamespacesIsMutable() {
         if (!((bitField0_ & 0x00000020) == 0x00000020)) {
-          namespaces_ = new java.util.ArrayList(namespaces_);
+          namespaces_ = new java.util.ArrayList(namespaces_);
           bitField0_ |= 0x00000020;
          }
       }
       /**
        * repeated bytes namespaces = 6;
        */
-      public java.util.List
+      public java.util.List
           getNamespacesList() {
         return java.util.Collections.unmodifiableList(namespaces_);
       }
@@ -6840,14 +6840,14 @@ public final class ZooKeeperProtos {
       /**
        * repeated bytes namespaces = 6;
        */
-      public com.google.protobuf.ByteString getNamespaces(int index) {
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index) {
         return namespaces_.get(index);
       }
       /**
        * repeated bytes namespaces = 6;
        */
       public Builder setNamespaces(
-          int index, com.google.protobuf.ByteString value) {
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -6859,7 +6859,7 @@ public final class ZooKeeperProtos {
       /**
        * repeated bytes namespaces = 6;
        */
-      public Builder addNamespaces(com.google.protobuf.ByteString value) {
+      public Builder addNamespaces(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -6872,9 +6872,9 @@ public final class ZooKeeperProtos {
        * repeated bytes namespaces = 6;
        */
       public Builder addAllNamespaces(
-          java.lang.Iterable values) {
+          java.lang.Iterable values) {
         ensureNamespacesIsMutable();
-        com.google.protobuf.AbstractMessageLite.Builder.addAll(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
             values, namespaces_);
         onChanged();
         return this;
@@ -6889,12 +6889,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -6912,22 +6912,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicationPeer parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicationPeer(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -6939,7 +6939,7 @@ public final class ZooKeeperProtos {
 
   public interface ReplicationStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required .hbase.pb.ReplicationState.State state = 1;
@@ -6959,11 +6959,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.ReplicationState}
    */
   public  static final class ReplicationState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationState)
       ReplicationStateOrBuilder {
     // Use ReplicationState.newBuilder() to construct.
-    private ReplicationState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicationState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicationState() {
@@ -6971,18 +6971,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicationState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7011,22 +7011,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7037,7 +7037,7 @@ public final class ZooKeeperProtos {
      * Protobuf enum {@code hbase.pb.ReplicationState.State}
      */
     public enum State
-        implements com.google.protobuf.ProtocolMessageEnum {
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
       /**
        * ENABLED = 0;
        */
@@ -7078,27 +7078,27 @@ public final class ZooKeeperProtos {
         }
       }
 
-      public static com.google.protobuf.Internal.EnumLiteMap
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap
           internalGetValueMap() {
         return internalValueMap;
       }
-      private static final com.google.protobuf.Internal.EnumLiteMap<
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
           State> internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap() {
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap() {
               public State findValueByNumber(int number) {
                 return State.forNumber(number);
               }
             };
 
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
           getValueDescriptor() {
         return getDescriptor().getValues().get(ordinal());
       }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptorForType() {
         return getDescriptor();
       }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState.getDescriptor().getEnumTypes().get(0);
       }
@@ -7106,7 +7106,7 @@ public final class ZooKeeperProtos {
       private static final State[] VALUES = values();
 
       public static State valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
         if (desc.getType() != getDescriptor()) {
           throw new java.lang.IllegalArgumentException(
             "EnumValueDescriptor is not for this type.");
@@ -7154,7 +7154,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeEnum(1, state_);
@@ -7168,7 +7168,7 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeEnumSize(1, state_);
       }
       size += unknownFields.getSerializedSize();
@@ -7213,61 +7213,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7285,7 +7285,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7298,15 +7298,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.ReplicationState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7319,12 +7319,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7335,7 +7335,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationState_descriptor;
       }
@@ -7369,29 +7369,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState)other);
         } else {
@@ -7418,13 +7418,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7472,12 +7472,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7495,22 +7495,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicationState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicationState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -7522,7 +7522,7 @@ public final class ZooKeeperProtos {
 
   public interface ReplicationHLogPositionOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationHLogPosition)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * required int64 position = 1;
@@ -7542,11 +7542,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.ReplicationHLogPosition}
    */
   public  static final class ReplicationHLogPosition extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationHLogPosition)
       ReplicationHLogPositionOrBuilder {
     // Use ReplicationHLogPosition.newBuilder() to construct.
-    private ReplicationHLogPosition(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private ReplicationHLogPosition(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private ReplicationHLogPosition() {
@@ -7554,18 +7554,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private ReplicationHLogPosition(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -7588,22 +7588,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationHLogPosition_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -7640,7 +7640,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, position_);
@@ -7654,7 +7654,7 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, position_);
       }
       size += unknownFields.getSerializedSize();
@@ -7692,7 +7692,7 @@ public final class ZooKeeperProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasPosition()) {
         hash = (37 * hash) + POSITION_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getPosition());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -7701,61 +7701,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -7773,7 +7773,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -7786,15 +7786,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.ReplicationHLogPosition}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationHLogPosition)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPositionOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationHLogPosition_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -7807,12 +7807,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -7823,7 +7823,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationHLogPosition_descriptor;
       }
@@ -7857,29 +7857,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition)other);
         } else {
@@ -7906,13 +7906,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationHLogPosition) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -7956,12 +7956,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -7979,22 +7979,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public ReplicationHLogPosition parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new ReplicationHLogPosition(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -8006,7 +8006,7 @@ public final class ZooKeeperProtos {
 
   public interface TableLockOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.TableLock)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional .hbase.pb.TableName table_name = 1;
@@ -8063,7 +8063,7 @@ public final class ZooKeeperProtos {
     /**
      * optional string purpose = 5;
      */
-    com.google.protobuf.ByteString
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getPurposeBytes();
 
     /**
@@ -8084,11 +8084,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.TableLock}
    */
   public  static final class TableLock extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.TableLock)
       TableLockOrBuilder {
     // Use TableLock.newBuilder() to construct.
-    private TableLock(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private TableLock(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private TableLock() {
@@ -8099,18 +8099,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private TableLock(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -8163,7 +8163,7 @@ public final class ZooKeeperProtos {
               break;
             }
             case 42: {
-              com.google.protobuf.ByteString bs = input.readBytes();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000010;
               purpose_ = bs;
               break;
@@ -8175,22 +8175,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableLock_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableLock_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -8286,8 +8286,8 @@ public final class ZooKeeperProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
           purpose_ = s;
@@ -8298,17 +8298,17 @@ public final class ZooKeeperProtos {
     /**
      * optional string purpose = 5;
      */
-    public com.google.protobuf.ByteString
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getPurposeBytes() {
       java.lang.Object ref = purpose_;
       if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         purpose_ = b;
         return b;
       } else {
-        return (com.google.protobuf.ByteString) ref;
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
       }
     }
 
@@ -8349,7 +8349,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getTableName());
@@ -8364,7 +8364,7 @@ public final class ZooKeeperProtos {
         output.writeBool(4, isShared_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        com.google.protobuf.GeneratedMessageV3.writeString(output, 5, purpose_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, purpose_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeInt64(6, createTime_);
@@ -8378,26 +8378,26 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getTableName());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, getLockOwner());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(3, threadId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, isShared_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, purpose_);
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, purpose_);
       }
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeInt64Size(6, createTime_);
       }
       size += unknownFields.getSerializedSize();
@@ -8468,12 +8468,12 @@ public final class ZooKeeperProtos {
       }
       if (hasThreadId()) {
         hash = (37 * hash) + THREAD_ID_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getThreadId());
       }
       if (hasIsShared()) {
         hash = (37 * hash) + IS_SHARED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIsShared());
       }
       if (hasPurpose()) {
@@ -8482,7 +8482,7 @@ public final class ZooKeeperProtos {
       }
       if (hasCreateTime()) {
         hash = (37 * hash) + CREATE_TIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getCreateTime());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -8491,61 +8491,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -8563,7 +8563,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -8576,15 +8576,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.TableLock}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.TableLock)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLockOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableLock_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableLock_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -8597,12 +8597,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getTableNameFieldBuilder();
           getLockOwnerFieldBuilder();
@@ -8633,7 +8633,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableLock_descriptor;
       }
@@ -8695,29 +8695,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock)other);
         } else {
@@ -8768,13 +8768,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableLock) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -8787,7 +8787,7 @@ public final class ZooKeeperProtos {
       private int bitField0_;
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * optional .hbase.pb.TableName table_name = 1;
@@ -8890,11 +8890,11 @@ public final class ZooKeeperProtos {
       /**
        * optional .hbase.pb.TableName table_name = 1;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
                   getTableName(),
                   getParentForChildren(),
@@ -8905,7 +8905,7 @@ public final class ZooKeeperProtos {
       }
 
       private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName lockOwner_ = null;
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> lockOwnerBuilder_;
       /**
        * optional .hbase.pb.ServerName lock_owner = 2;
@@ -9008,11 +9008,11 @@ public final class ZooKeeperProtos {
       /**
        * optional .hbase.pb.ServerName lock_owner = 2;
        */
-      private com.google.protobuf.SingleFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
           getLockOwnerFieldBuilder() {
         if (lockOwnerBuilder_ == null) {
-          lockOwnerBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+          lockOwnerBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
                   getLockOwner(),
                   getParentForChildren(),
@@ -9099,8 +9099,8 @@ public final class ZooKeeperProtos {
       public java.lang.String getPurpose() {
         java.lang.Object ref = purpose_;
         if (!(ref instanceof java.lang.String)) {
-          com.google.protobuf.ByteString bs =
-              (com.google.protobuf.ByteString) ref;
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
             purpose_ = s;
@@ -9113,17 +9113,17 @@ public final class ZooKeeperProtos {
       /**
        * optional string purpose = 5;
        */
-      public com.google.protobuf.ByteString
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
           getPurposeBytes() {
         java.lang.Object ref = purpose_;
         if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           purpose_ = b;
           return b;
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         }
       }
       /**
@@ -9152,7 +9152,7 @@ public final class ZooKeeperProtos {
        * optional string purpose = 5;
        */
       public Builder setPurposeBytes(
-          com.google.protobuf.ByteString value) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
@@ -9194,12 +9194,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9217,22 +9217,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public TableLock parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new TableLock(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9244,7 +9244,7 @@ public final class ZooKeeperProtos {
 
   public interface SwitchStateOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SwitchState)
-      com.google.protobuf.MessageOrBuilder {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
      * optional bool enabled = 1;
@@ -9264,11 +9264,11 @@ public final class ZooKeeperProtos {
    * Protobuf type {@code hbase.pb.SwitchState}
    */
   public  static final class SwitchState extends
-      com.google.protobuf.GeneratedMessageV3 implements
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:hbase.pb.SwitchState)
       SwitchStateOrBuilder {
     // Use SwitchState.newBuilder() to construct.
-    private SwitchState(com.google.protobuf.GeneratedMessageV3.Builder builder) {
+    private SwitchState(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
     private SwitchState() {
@@ -9276,18 +9276,18 @@ public final class ZooKeeperProtos {
     }
 
     @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
     getUnknownFields() {
       return this.unknownFields;
     }
     private SwitchState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       this();
       int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
         boolean done = false;
         while (!done) {
@@ -9310,22 +9310,22 @@ public final class ZooKeeperProtos {
             }
           }
         }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(this);
       } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
-    public static final com.google.protobuf.Descriptors.Descriptor
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
     }
 
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
       return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
@@ -9358,7 +9358,7 @@ public final class ZooKeeperProtos {
       return true;
     }
 
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(1, enabled_);
@@ -9372,7 +9372,7 @@ public final class ZooKeeperProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(1, enabled_);
       }
       size += unknownFields.getSerializedSize();
@@ -9410,7 +9410,7 @@ public final class ZooKeeperProtos {
       hash = (19 * hash) + getDescriptorForType().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getEnabled());
       }
       hash = (29 * hash) + unknownFields.hashCode();
@@ -9419,61 +9419,61 @@ public final class ZooKeeperProtos {
     }
 
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
         byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseDelimitedFrom(
         java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.CodedInputStream input)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
     public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
 
@@ -9491,7 +9491,7 @@ public final class ZooKeeperProtos {
 
     @java.lang.Override
     protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
       Builder builder = new Builder(parent);
       return builder;
     }
@@ -9504,15 +9504,15 @@ public final class ZooKeeperProtos {
      * Protobuf type {@code hbase.pb.SwitchState}
      */
     public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder implements
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:hbase.pb.SwitchState)
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
       }
 
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
@@ -9525,12 +9525,12 @@ public final class ZooKeeperProtos {
       }
 
       private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
         super(parent);
         maybeForceBuilderInitialization();
       }
       private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
         }
       }
@@ -9541,7 +9541,7 @@ public final class ZooKeeperProtos {
         return this;
       }
 
-      public com.google.protobuf.Descriptors.Descriptor
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
         return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
       }
@@ -9575,29 +9575,29 @@ public final class ZooKeeperProtos {
         return (Builder) super.clone();
       }
       public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.setField(field, value);
       }
       public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
         return (Builder) super.clearField(field);
       }
       public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
         return (Builder) super.clearOneof(oneof);
       }
       public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           int index, Object value) {
         return (Builder) super.setRepeatedField(field, index, value);
       }
       public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
           Object value) {
         return (Builder) super.addRepeatedField(field, value);
       }
-      public Builder mergeFrom(com.google.protobuf.Message other) {
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
         if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState) {
           return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState)other);
         } else {
@@ -9621,13 +9621,13 @@ public final class ZooKeeperProtos {
       }
 
       public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
         org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
           parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
@@ -9671,12 +9671,12 @@ public final class ZooKeeperProtos {
         return this;
       }
       public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
       }
 
       public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.mergeUnknownFields(unknownFields);
       }
 
@@ -9694,22 +9694,22 @@ public final class ZooKeeperProtos {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final com.google.protobuf.Parser
-        PARSER = new com.google.protobuf.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() {
       public SwitchState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
           return new SwitchState(input, extensionRegistry);
       }
     };
 
-    public static com.google.protobuf.Parser parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public com.google.protobuf.Parser getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() {
       return PARSER;
     }
 
@@ -9719,67 +9719,67 @@ public final class ZooKeeperProtos {
 
   }
 
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MetaRegionServer_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_MetaRegionServer_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Master_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Master_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ClusterUp_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ClusterUp_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SplitLogTask_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SplitLogTask_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_DeprecatedTableState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableCF_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TableCF_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationPeer_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicationState_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ReplicationHLogPosition_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableLock_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TableLock_fieldAccessorTable;
-  private static final com.google.protobuf.Descriptors.Descriptor
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SwitchState_descriptor;
   private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SwitchState_fieldAccessorTable;
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
+  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
     return descriptor;
   }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
+  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       descriptor;
   static {
     java.lang.String[] descriptorData = {
@@ -9821,84 +9821,84 @@ public final class ZooKeeperProtos {
       "BL\n1org.apache.hadoop.hbase.shaded.proto" +
       "buf.generatedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
     };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-        new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
-          public com.google.protobuf.ExtensionRegistry assignDescriptors(
-              com.google.protobuf.Descriptors.FileDescriptor root) {
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
+          public org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry assignDescriptors(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor root) {
             descriptor = root;
             return null;
           }
         };
-    com.google.protobuf.Descriptors.FileDescriptor
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_MetaRegionServer_descriptor =
       getDescriptor().getMessageTypes().get(0);
     internal_static_hbase_pb_MetaRegionServer_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_MetaRegionServer_descriptor,
         new java.lang.String[] { "Server", "RpcVersion", "State", });
     internal_static_hbase_pb_Master_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_Master_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Master_descriptor,
         new java.lang.String[] { "Master", "RpcVersion", "InfoPort", });
     internal_static_hbase_pb_ClusterUp_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_ClusterUp_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ClusterUp_descriptor,
         new java.lang.String[] { "StartDate", });
     internal_static_hbase_pb_SplitLogTask_descriptor =
       getDescriptor().getMessageTypes().get(3);
     internal_static_hbase_pb_SplitLogTask_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SplitLogTask_descriptor,
         new java.lang.String[] { "State", "ServerName", "Mode", });
     internal_static_hbase_pb_DeprecatedTableState_descriptor =
       getDescriptor().getMessageTypes().get(4);
     internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_DeprecatedTableState_descriptor,
         new java.lang.String[] { "State", });
     internal_static_hbase_pb_TableCF_descriptor =
       getDescriptor().getMessageTypes().get(5);
     internal_static_hbase_pb_TableCF_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TableCF_descriptor,
         new java.lang.String[] { "TableName", "Families", });
     internal_static_hbase_pb_ReplicationPeer_descriptor =
       getDescriptor().getMessageTypes().get(6);
     internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicationPeer_descriptor,
         new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", "TableCfs", "Namespaces", });
     internal_static_hbase_pb_ReplicationState_descriptor =
       getDescriptor().getMessageTypes().get(7);
     internal_static_hbase_pb_ReplicationState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicationState_descriptor,
         new java.lang.String[] { "State", });
     internal_static_hbase_pb_ReplicationHLogPosition_descriptor =
       getDescriptor().getMessageTypes().get(8);
     internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReplicationHLogPosition_descriptor,
         new java.lang.String[] { "Position", });
     internal_static_hbase_pb_TableLock_descriptor =
       getDescriptor().getMessageTypes().get(9);
     internal_static_hbase_pb_TableLock_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_TableLock_descriptor,
         new java.lang.String[] { "TableName", "LockOwner", "ThreadId", "IsShared", "Purpose", "CreateTime", });
     internal_static_hbase_pb_SwitchState_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_hbase_pb_SwitchState_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_SwitchState_descriptor,
         new java.lang.String[] { "Enabled", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/util/ByteStringer.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/util/ByteStringer.java
deleted file mode 100644
index 38cc0bd..0000000
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/util/ByteStringer.java
+++ /dev/null
@@ -1,48 +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.shaded.util;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.UnsafeByteOperations;
-
-/**
- * Hack to workaround HBASE-10304 issue that keeps bubbling up when a mapreduce context.
- */
-// Depends on protobuf-3.1.0 feature.
-@InterfaceAudience.Private
-public class ByteStringer {
-  private ByteStringer() {
-    super();
-  }
-
-  /**
-   * Wraps a byte array in a {@link ByteString} without copying it.
-   */
-  public static ByteString wrap(final byte[] array) {
-    return UnsafeByteOperations.unsafeWrap(array);
-  }
-
-  /**
-   * Wraps a subset of a byte array in a {@link ByteString} without copying it.
-   */
-  public static ByteString wrap(final byte[] array, int offset, int length) {
-    return UnsafeByteOperations.unsafeWrap(array, offset, length);
-  }
-}
diff --git a/hbase-protocol-shaded/src/main/protobuf/RSGroup.proto b/hbase-protocol-shaded/src/main/protobuf/RSGroup.proto
deleted file mode 100644
index d65cad4..0000000
--- a/hbase-protocol-shaded/src/main/protobuf/RSGroup.proto
+++ /dev/null
@@ -1,34 +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 hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
-option java_outer_classname = "RSGroupProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message RSGroupInfo {
-  required string name = 1;
-  repeated ServerName servers = 4;
-  repeated TableName tables = 3;
-}
-
diff --git a/hbase-protocol-shaded/src/main/protobuf/RSGroupAdmin.proto b/hbase-protocol-shaded/src/main/protobuf/RSGroupAdmin.proto
deleted file mode 100644
index 550cdcb..0000000
--- a/hbase-protocol-shaded/src/main/protobuf/RSGroupAdmin.proto
+++ /dev/null
@@ -1,136 +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 hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
-option java_outer_classname = "RSGroupAdminProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "RSGroup.proto";
-
-/** Group level protobufs */
-
-message ListTablesOfRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message ListTablesOfRSGroupResponse {
-  repeated TableName table_name = 1;
-}
-
-message GetRSGroupInfoRequest {
-  required string r_s_group_name = 1;
-}
-
-message GetRSGroupInfoResponse {
-  optional RSGroupInfo r_s_group_info = 1;
-}
-
-message GetRSGroupInfoOfTableRequest {
-  required TableName table_name = 1;
-}
-
-message GetRSGroupInfoOfTableResponse {
-  optional RSGroupInfo r_s_group_info = 1;
-}
-
-message MoveServersRequest {
-  required string target_group = 1;
-  repeated ServerName servers = 3;
-}
-
-message MoveServersResponse {
-}
-
-message MoveTablesRequest {
-  required string target_group = 1;
-  repeated TableName table_name = 2;
-}
-
-message MoveTablesResponse {
-}
-
-message AddRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message AddRSGroupResponse {
-}
-
-message RemoveRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message RemoveRSGroupResponse {
-}
-
-message BalanceRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message BalanceRSGroupResponse {
-  required bool balanceRan = 1;
-}
-
-message ListRSGroupInfosRequest {
-}
-
-message ListRSGroupInfosResponse {
-  repeated RSGroupInfo  r_s_group_info = 1;
-}
-
-message GetRSGroupInfoOfServerRequest {
-  required ServerName server = 2;
-}
-
-message GetRSGroupInfoOfServerResponse {
-  optional RSGroupInfo r_s_group_info = 1;
-}
-
-service RSGroupAdminService {
-  rpc GetRSGroupInfo(GetRSGroupInfoRequest)
-    returns (GetRSGroupInfoResponse);
-
-  rpc GetRSGroupInfoOfTable(GetRSGroupInfoOfTableRequest)
-    returns (GetRSGroupInfoOfTableResponse);
-
-  rpc GetRSGroupInfoOfServer(GetRSGroupInfoOfServerRequest)
-    returns (GetRSGroupInfoOfServerResponse);
-
-  rpc MoveServers(MoveServersRequest)
-    returns (MoveServersResponse);
-
-  rpc MoveTables(MoveTablesRequest)
-    returns (MoveTablesResponse);
-
-  rpc AddRSGroup(AddRSGroupRequest)
-    returns (AddRSGroupResponse);
-
-  rpc RemoveRSGroup(RemoveRSGroupRequest)
-    returns (RemoveRSGroupResponse);
-
-  rpc BalanceRSGroup(BalanceRSGroupRequest)
-    returns (BalanceRSGroupResponse);
-
-  rpc ListRSGroupInfos(ListRSGroupInfosRequest)
-    returns (ListRSGroupInfosResponse);
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
index 9ce3a33..ea162fc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
@@ -22,12 +22,12 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos;
 
 /**
@@ -47,15 +47,16 @@ public class MessageCodec implements Codec {
       CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
       // This copies bytes from Cell to ByteString.  I don't see anyway around the copy.
       // ByteString is final.
-      builder.setRow(ByteStringer.wrap(cell.getRowArray(), cell.getRowOffset(),
+      builder.setRow(UnsafeByteOperations.unsafeWrap(cell.getRowArray(), cell.getRowOffset(),
           cell.getRowLength()));
-      builder.setFamily(ByteStringer.wrap(cell.getFamilyArray(), cell.getFamilyOffset(),
+      builder.setFamily(UnsafeByteOperations.unsafeWrap(cell.getFamilyArray(),
+          cell.getFamilyOffset(),
           cell.getFamilyLength()));
-      builder.setQualifier(ByteStringer.wrap(cell.getQualifierArray(),
+      builder.setQualifier(UnsafeByteOperations.unsafeWrap(cell.getQualifierArray(),
           cell.getQualifierOffset(), cell.getQualifierLength()));
       builder.setTimestamp(cell.getTimestamp());
       builder.setCellType(CellProtos.CellType.valueOf(cell.getTypeByte()));
-      builder.setValue(ByteStringer.wrap(cell.getValueArray(), cell.getValueOffset(),
+      builder.setValue(UnsafeByteOperations.unsafeWrap(cell.getValueArray(), cell.getValueOffset(),
           cell.getValueLength()));
       CellProtos.Cell pbcell = builder.build();
       pbcell.writeDelimitedTo(this.out);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java
index 4d36fc4..62fd3fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java
@@ -25,12 +25,12 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
 
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.KeyValueUtil;
+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.FSProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -196,7 +196,7 @@ public class Reference {
     FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
     builder.setRange(isTopFileRegion(getFileRegion())?
       FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM);
-    builder.setSplitkey(ByteStringer.wrap(getSplitKey()));
+    builder.setSplitkey(UnsafeByteOperations.unsafeWrap(getSplitKey()));
     return builder.build();
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index df39ea5..7eac9c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -26,13 +26,13 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
-import org.apache.hadoop.hbase.CellComparator.MetaCellComparator;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellComparator.MetaCellComparator;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -207,7 +207,7 @@ public class FixedFileTrailer {
       .setComparatorClassName(comparatorClassName)
       .setCompressionCodec(compressionCodec.ordinal());
     if (encryptionKey != null) {
-      builder.setEncryptionKey(ByteStringer.wrap(encryptionKey));
+      builder.setEncryptionKey(UnsafeByteOperations.unsafeWrap(encryptionKey));
     }
     // We need this extra copy unfortunately to determine the final size of the
     // delimited output, see use of baos.size() below.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 5fd2de8..d3669f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -18,9 +18,6 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
 import java.io.ByteArrayInputStream;
 import java.io.Closeable;
 import java.io.DataInput;
@@ -39,7 +36,6 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.LongAdder;
 
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -58,6 +54,7 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
+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.BytesBytesPair;
@@ -67,6 +64,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.io.Writable;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
 /**
  * File format for hbase.
  * A file of sorted key/value pairs. Both keys and values are byte arrays.
@@ -746,8 +746,8 @@ public class HFile {
       HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
       for (Map.Entry e: this.map.entrySet()) {
         HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
-        bbpBuilder.setFirst(ByteStringer.wrap(e.getKey()));
-        bbpBuilder.setSecond(ByteStringer.wrap(e.getValue()));
+        bbpBuilder.setFirst(UnsafeByteOperations.unsafeWrap(e.getKey()));
+        bbpBuilder.setSecond(UnsafeByteOperations.unsafeWrap(e.getValue()));
         builder.addMapEntry(bbpBuilder.build());
       }
       out.write(ProtobufMagic.PB_MAGIC);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 3854fa6..11788fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -90,7 +90,7 @@ import org.apache.hadoop.hbase.security.visibility.VisibilityController;
 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.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -919,7 +919,7 @@ public class MasterRpcServices extends RSRpcServices
           builder.setException(ForeignExceptionUtil.toProtoForeignException(result.getException()));
         }
         if (result.hasResultData()) {
-          builder.setResult(ByteStringer.wrap(result.getResult()));
+          builder.setResult(UnsafeByteOperations.unsafeWrap(result.getResult()));
         }
         master.getMasterProcedureExecutor().removeResult(request.getProcId());
       } else {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
index aa618db..238ecf2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
@@ -32,10 +32,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+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.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -176,7 +176,7 @@ public class DeleteColumnFamilyProcedure
         MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
             .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
             .setTableName(ProtobufUtil.toProtoTableName(tableName))
-            .setColumnfamilyName(ByteStringer.wrap(familyName));
+            .setColumnfamilyName(UnsafeByteOperations.unsafeWrap(familyName));
     if (unmodifiedHTableDescriptor != null) {
       deleteCFMsg
           .setUnmodifiedTableSchema(ProtobufUtil.convertToTableSchema(unmodifiedHTableDescriptor));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index ba93a17..c301759 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -37,12 +37,12 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -112,10 +112,10 @@ public class ReplicationProtbufUtil {
       WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
       WALKey key = entry.getKey();
       keyBuilder.setEncodedRegionName(
-        ByteStringer.wrap(encodedRegionName == null
+          UnsafeByteOperations.unsafeWrap(encodedRegionName == null
             ? key.getEncodedRegionName()
             : encodedRegionName));
-      keyBuilder.setTableName(ByteStringer.wrap(key.getTablename().getName()));
+      keyBuilder.setTableName(UnsafeByteOperations.unsafeWrap(key.getTablename().getName()));
       keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
       keyBuilder.setWriteTime(key.getWriteTime());
       if (key.getNonce() != HConstants.NO_NONCE) {
@@ -136,7 +136,7 @@ public class ReplicationProtbufUtil {
       NavigableMap scopes = key.getReplicationScopes();
       if (scopes != null && !scopes.isEmpty()) {
         for (Map.Entry scope: scopes.entrySet()) {
-          scopeBuilder.setFamily(ByteStringer.wrap(scope.getKey()));
+          scopeBuilder.setFamily(UnsafeByteOperations.unsafeWrap(scope.getKey()));
           WALProtos.ScopeType scopeType =
               WALProtos.ScopeType.valueOf(scope.getValue().intValue());
           scopeBuilder.setScopeType(scopeType);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 565c1c0..07d1fa5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -129,21 +129,6 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
-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.regionserver.MultiVersionConcurrencyControl.WriteEntry;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
@@ -158,9 +143,24 @@ import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 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.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
+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.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -5553,7 +5553,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         try {
           WALProtos.BulkLoadDescriptor loadDescriptor =
               ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
-                ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles,
+                  UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()),
+                  storeFiles,
                 storeFilesSizes, seqId);
           WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
               loadDescriptor, mvcc);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index a3d0a4b..b07a40c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -127,11 +127,12 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 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;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+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.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
@@ -152,7 +153,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -1537,7 +1537,7 @@ public class HRegionServer extends HasThread implements
       regionSpecifier = RegionSpecifier.newBuilder();
     }
     regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
-    regionSpecifier.setValue(ByteStringer.wrap(name));
+    regionSpecifier.setValue(UnsafeByteOperations.unsafeWrap(name));
     regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
       .setStores(stores)
       .setStorefiles(storefiles)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
index 569a16a..55c4771 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
@@ -41,11 +41,11 @@ import org.apache.hadoop.hbase.io.crypto.Cipher;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.crypto.Encryptor;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
 import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.hadoop.hbase.util.FSUtils;
 
@@ -112,7 +112,7 @@ public abstract class AbstractProtobufLogWriter {
       byte[] keyBytes = new byte[cipher.getKeyLength()];
       rng.nextBytes(keyBytes);
       Key key = new SecretKeySpec(keyBytes, cipher.getName());
-      builder.setEncryptionKey(ByteStringer.wrap(EncryptionUtil.wrapKey(conf,
+      builder.setEncryptionKey(UnsafeByteOperations.unsafeWrap(EncryptionUtil.wrapKey(conf,
           conf.get(HConstants.CRYPTO_WAL_KEY_NAME_CONF_KEY,
               conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
                   User.getCurrent().getShortName())),
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
index 93c8cfa..cceeebc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
@@ -24,23 +24,23 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.Callable;
-import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorCompletionService;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 
@@ -184,7 +184,7 @@ public final class SnapshotManifestV1 {
         // 2.1. build the snapshot reference for the store
         SnapshotRegionManifest.FamilyFiles.Builder family =
               SnapshotRegionManifest.FamilyFiles.newBuilder();
-        family.setFamilyName(ByteStringer.wrap(Bytes.toBytes(familyName)));
+        family.setFamilyName(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(familyName)));
 
         if (LOG.isDebugEnabled()) {
           LOG.debug("Adding snapshot references for " + storeFiles  + " hfiles");
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
index 36e5121..18c2a13 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -42,7 +44,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
@@ -101,7 +102,7 @@ public final class SnapshotManifestV2 {
         final SnapshotRegionManifest.Builder region, final byte[] familyName) {
       SnapshotRegionManifest.FamilyFiles.Builder family =
           SnapshotRegionManifest.FamilyFiles.newBuilder();
-      family.setFamilyName(ByteStringer.wrap(familyName));
+      family.setFamilyName(UnsafeByteOperations.unsafeWrap(familyName));
       return family;
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
index fea65f5..cd76b17 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 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.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -245,11 +245,11 @@ public class TestHRegionInfo {
     // test convert RegionInfo without replicaId
     RegionInfo info = RegionInfo.newBuilder()
       .setTableName(HBaseProtos.TableName.newBuilder()
-        .setQualifier(ByteStringer.wrap(tableName.getQualifier()))
-        .setNamespace(ByteStringer.wrap(tableName.getNamespace()))
+        .setQualifier(UnsafeByteOperations.unsafeWrap(tableName.getQualifier()))
+        .setNamespace(UnsafeByteOperations.unsafeWrap(tableName.getNamespace()))
         .build())
-      .setStartKey(ByteStringer.wrap(startKey))
-      .setEndKey(ByteStringer.wrap(endKey))
+      .setStartKey(UnsafeByteOperations.unsafeWrap(startKey))
+      .setEndKey(UnsafeByteOperations.unsafeWrap(endKey))
       .setSplit(split)
       .setRegionId(regionId)
       .build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index e1a2486..bcdbff6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
+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.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
@@ -78,7 +79,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
 import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -1133,11 +1133,13 @@ public class TestHRegionReplayEvents {
     putDataByReplay(secondaryRegion, 0, 10, cq, families);
     secondaryRegion.replayWALFlushStartMarker(FlushDescriptor.newBuilder().
       setFlushSequenceNumber(10)
-      .setTableName(ByteStringer.wrap(primaryRegion.getTableDesc().getTableName().getName()))
+      .setTableName(UnsafeByteOperations.unsafeWrap(
+          primaryRegion.getTableDesc().getTableName().getName()))
       .setAction(FlushAction.START_FLUSH)
       .setEncodedRegionName(
-        ByteStringer.wrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
-      .setRegionName(ByteStringer.wrap(primaryRegion.getRegionInfo().getRegionName()))
+          UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
+      .setRegionName(UnsafeByteOperations.unsafeWrap(
+          primaryRegion.getRegionInfo().getRegionName()))
       .build());
 
     verify(walSecondary, times(0)).append((HRegionInfo)any(),
@@ -1541,13 +1543,14 @@ public class TestHRegionReplayEvents {
     // from primary and also deleted from the archive directory
     secondaryRegion.replayWALFlushCommitMarker(FlushDescriptor.newBuilder().
       setFlushSequenceNumber(Long.MAX_VALUE)
-      .setTableName(ByteStringer.wrap(primaryRegion.getTableDesc().getTableName().getName()))
+      .setTableName(UnsafeByteOperations.unsafeWrap(primaryRegion.getTableDesc().getTableName().getName()))
       .setAction(FlushAction.COMMIT_FLUSH)
       .setEncodedRegionName(
-        ByteStringer.wrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
-      .setRegionName(ByteStringer.wrap(primaryRegion.getRegionInfo().getRegionName()))
+          UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
+      .setRegionName(UnsafeByteOperations.unsafeWrap(
+          primaryRegion.getRegionInfo().getRegionName()))
       .addStoreFlushes(StoreFlushDescriptor.newBuilder()
-        .setFamilyName(ByteStringer.wrap(families[0]))
+        .setFamilyName(UnsafeByteOperations.unsafeWrap(families[0]))
         .setStoreHomeDir("/store_home_dir")
         .addFlushOutput("/foo/baz/bar")
         .build())
@@ -1559,14 +1562,15 @@ public class TestHRegionReplayEvents {
     // tests replaying compaction marker, but the compaction output file has already been compacted
     // from primary and also deleted from the archive directory
     secondaryRegion.replayWALCompactionMarker(CompactionDescriptor.newBuilder()
-      .setTableName(ByteStringer.wrap(primaryRegion.getTableDesc().getTableName().getName()))
+      .setTableName(UnsafeByteOperations.unsafeWrap(
+          primaryRegion.getTableDesc().getTableName().getName()))
       .setEncodedRegionName(
-        ByteStringer.wrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
-      .setFamilyName(ByteStringer.wrap(families[0]))
+          UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
+      .setFamilyName(UnsafeByteOperations.unsafeWrap(families[0]))
       .addCompactionInput("/foo")
       .addCompactionOutput("/bar")
       .setStoreHomeDir("/store_home_dir")
-      .setRegionName(ByteStringer.wrap(primaryRegion.getRegionInfo().getRegionName()))
+      .setRegionName(UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getRegionName()))
       .build()
       , true, true, Long.MAX_VALUE);
   }
@@ -1576,15 +1580,16 @@ public class TestHRegionReplayEvents {
     // tests replaying region open event marker, but the region files have already been compacted
     // from primary and also deleted from the archive directory
     secondaryRegion.replayWALRegionEventMarker(RegionEventDescriptor.newBuilder()
-      .setTableName(ByteStringer.wrap(primaryRegion.getTableDesc().getTableName().getName()))
+      .setTableName(UnsafeByteOperations.unsafeWrap(
+          primaryRegion.getTableDesc().getTableName().getName()))
       .setEncodedRegionName(
-        ByteStringer.wrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
-      .setRegionName(ByteStringer.wrap(primaryRegion.getRegionInfo().getRegionName()))
+          UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
+      .setRegionName(UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getRegionName()))
       .setEventType(EventType.REGION_OPEN)
       .setServer(ProtobufUtil.toServerName(ServerName.valueOf("foo", 1, 1)))
       .setLogSequenceNumber(Long.MAX_VALUE)
       .addStores(StoreDescriptor.newBuilder()
-        .setFamilyName(ByteStringer.wrap(families[0]))
+        .setFamilyName(UnsafeByteOperations.unsafeWrap(families[0]))
         .setStoreHomeDir("/store_home_dir")
         .addStoreFile("/foo")
         .build())
@@ -1598,10 +1603,10 @@ public class TestHRegionReplayEvents {
     secondaryRegion.replayWALBulkLoadEventMarker(BulkLoadDescriptor.newBuilder()
       .setTableName(ProtobufUtil.toProtoTableName(primaryRegion.getTableDesc().getTableName()))
       .setEncodedRegionName(
-        ByteStringer.wrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
+          UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
       .setBulkloadSeqNum(Long.MAX_VALUE)
       .addStores(StoreDescriptor.newBuilder()
-        .setFamilyName(ByteStringer.wrap(families[0]))
+        .setFamilyName(UnsafeByteOperations.unsafeWrap(families[0]))
         .setStoreHomeDir("/store_home_dir")
         .addStoreFile("/foo")
         .build())
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
index ed77e11..4c4bae4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
@@ -27,7 +27,6 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -48,6 +47,7 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 
 /**
  * Tests that verify certain RPCs get a higher QoS.
@@ -80,12 +80,13 @@ public class TestPriorityRpc {
     GetRequest.Builder getRequestBuilder = GetRequest.newBuilder();
     RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
     regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
-    ByteString name = ByteStringer.wrap(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
+    ByteString name = UnsafeByteOperations.unsafeWrap(
+        HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
     regionSpecifierBuilder.setValue(name);
     RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
     getRequestBuilder.setRegion(regionSpecifier);
     Get.Builder getBuilder = Get.newBuilder();
-    getBuilder.setRow(ByteStringer.wrap("somerow".getBytes()));
+    getBuilder.setRow(UnsafeByteOperations.unsafeWrap("somerow".getBytes()));
     getRequestBuilder.setGet(getBuilder.build());
     GetRequest getRequest = getRequestBuilder.build();
     RequestHeader header = headerBuilder.build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 90e2d80..62a94f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -57,13 +57,13 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID;
+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.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
@@ -309,7 +309,8 @@ public class TestReplicationSink {
       HRegionInfo regionInfo = l.getAllRegionLocations().get(0).getRegionInfo();
       loadDescriptor =
           ProtobufUtil.toBulkLoadDescriptor(TABLE_NAME1,
-            ByteStringer.wrap(regionInfo.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
+              UnsafeByteOperations.unsafeWrap(regionInfo.getEncodedNameAsBytes()),
+              storeFiles, storeFilesSize, 1);
       edit = WALEdit.createBulkLoadEvent(regionInfo, loadDescriptor);
     }
     List entries = new ArrayList(1);
@@ -386,9 +387,9 @@ public class TestReplicationSink {
     uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
     uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
     keyBuilder.setClusterId(uuidBuilder.build());
-    keyBuilder.setTableName(ByteStringer.wrap(table.getName()));
+    keyBuilder.setTableName(UnsafeByteOperations.unsafeWrap(table.getName()));
     keyBuilder.setWriteTime(System.currentTimeMillis());
-    keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY));
+    keyBuilder.setEncodedRegionName(UnsafeByteOperations.unsafeWrap(HConstants.EMPTY_BYTE_ARRAY));
     keyBuilder.setLogSequenceNumber(-1);
     builder.setKey(keyBuilder.build());
     return builder;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 44821e9..7174d5f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+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.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
@@ -72,7 +73,6 @@ import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -492,7 +492,7 @@ public abstract class TestReplicationSourceManager {
     // 2. Create bulk load descriptor
     BulkLoadDescriptor desc =
         ProtobufUtil.toBulkLoadDescriptor(hri.getTable(),
-          ByteStringer.wrap(hri.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
+      UnsafeByteOperations.unsafeWrap(hri.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
 
     // 3. create bulk load wal edit event
     WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java
index 9eeb08c..2a610bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java
@@ -31,11 +31,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
+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.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -87,7 +87,7 @@ public class TestSnapshotManifest {
       for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
         SnapshotRegionManifest.FamilyFiles.Builder family =
             SnapshotRegionManifest.FamilyFiles.newBuilder();
-        family.setFamilyName(ByteStringer.wrap(hcd.getName()));
+        family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
         for (int j = 0; j < 100; ++j) {
           SnapshotRegionManifest.StoreFile.Builder sfManifest =
             SnapshotRegionManifest.StoreFile.newBuilder();
diff --git a/pom.xml b/pom.xml
index 08f5cb3..320486d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -655,6 +655,7 @@
             
           
           
+            true
             
               hbase-site.xml
               hdfs-site.xml
@@ -2839,7 +2840,7 @@
                 
                     org.apache.maven.plugins
                     maven-compiler-plugin
-                    3.1
+                    3.5.1
                     
                         javac-with-errorprone
                         true