You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2017/01/06 02:12:37 UTC

[1/4] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Repository: hbase
Updated Branches:
  refs/heads/master 0f6c79eb1 -> e02ae7724


http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/protobuf/Replication.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
index 05b6e59..c748514 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -24,7 +24,45 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
-import "ZooKeeper.proto";
+import "HBase.proto";
+
+message TableCF {
+  optional TableName table_name = 1;
+  repeated bytes families = 2;
+}
+
+/**
+ * Used by replication. Holds a replication peer key.
+ */
+message ReplicationPeer {
+  // clusterkey is the concatenation of the slave cluster's
+  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+  required string clusterkey = 1;
+  optional string replicationEndpointImpl = 2;
+  repeated BytesBytesPair data = 3;
+  repeated NameStringPair configuration = 4;
+  repeated TableCF table_cfs = 5;
+  repeated bytes namespaces = 6;
+  optional int64 bandwidth = 7;
+}
+
+/**
+ * Used by replication. Holds whether enabled or disabled
+ */
+message ReplicationState {
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+  }
+  required State state = 1;
+}
+
+/**
+ * Used by replication. Holds the current position in an WAL file.
+ */
+message ReplicationHLogPosition {
+  required int64 position = 1;
+}
 
 message AddReplicationPeerRequest {
   required string peer_id = 1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
index 323862c..b3bd2ec 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
@@ -105,44 +105,6 @@ message DeprecatedTableState {
   required State state = 1 [default = ENABLED];
 }
 
-message TableCF {
-  optional TableName table_name = 1;
-  repeated bytes families = 2;
-}
-
-/**
- * Used by replication. Holds a replication peer key.
- */
-message ReplicationPeer {
-  // clusterkey is the concatenation of the slave cluster's
-  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-  required string clusterkey = 1;
-  optional string replicationEndpointImpl = 2;
-  repeated BytesBytesPair data = 3;
-  repeated NameStringPair configuration = 4;
-  repeated TableCF table_cfs = 5;
-  repeated bytes namespaces = 6;
-  optional int64 bandwidth = 7;
-}
-
-/**
- * Used by replication. Holds whether enabled or disabled
- */
-message ReplicationState {
-  enum State {
-    ENABLED = 0;
-    DISABLED = 1;
-  }
-  required State state = 1;
-}
-
-/**
- * Used by replication. Holds the current position in an WAL file.
- */
-message ReplicationHLogPosition {
-  required int64 position = 1;
-}
-
 /**
  * Metadata associated with a table lock in zookeeper
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
index 1494892..08888f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -78,7 +78,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
         if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().size() == 0) {
           // we copy TableCFs node into PeerNode
           LOG.info("copy tableCFs into peerNode:" + peerId);
-          ZooKeeperProtos.TableCF[] tableCFs =
+          ReplicationProtos.TableCF[] tableCFs =
                   ReplicationSerDeHelper.parseTableCFs(
                           ZKUtil.getData(this.zookeeper, tableCFsNode));
           if (tableCFs != null && tableCFs.length > 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
index 8f52cbe..bd70be0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -271,7 +271,7 @@ public class TestPerTableCFReplication {
   @Test
   public void testTableCFsHelperConverter() {
 
-    ZooKeeperProtos.TableCF[] tableCFs = null;
+    ReplicationProtos.TableCF[] tableCFs = null;
     Map<TableName, List<String>> tabCFsMap = null;
 
     // 1. null or empty string, result should be null


[4/4] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by zg...@apache.org.
HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e02ae772
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e02ae772
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e02ae772

Branch: refs/heads/master
Commit: e02ae7724ddaa147a7cf41dc398e09e456e0dad6
Parents: 0f6c79e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Jan 4 17:48:09 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Jan 6 10:01:22 2017 +0800

----------------------------------------------------------------------
 .../replication/ReplicationSerDeHelper.java     |   50 +-
 .../replication/ReplicationPeerZKImpl.java      |   14 +-
 .../replication/ReplicationPeersZKImpl.java     |   10 +-
 .../replication/ReplicationStateZKBase.java     |   14 +-
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |   18 +-
 .../protobuf/generated/ReplicationProtos.java   | 4234 +++++++++++++++++-
 .../protobuf/generated/ZooKeeperProtos.java     | 4048 +----------------
 .../src/main/protobuf/Replication.proto         |   40 +-
 .../src/main/protobuf/ZooKeeper.proto           |   38 -
 .../replication/master/TableCFsUpdater.java     |    4 +-
 .../replication/TestPerTableCFReplication.java  |    4 +-
 11 files changed, 4237 insertions(+), 4237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
index dd83fb1..9e04c9b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
@@ -29,7 +29,7 @@ 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.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
@@ -62,13 +62,13 @@ public final class ReplicationSerDeHelper {
   }
 
   /** convert map to TableCFs Object */
-  public static ZooKeeperProtos.TableCF[] convert(
+  public static ReplicationProtos.TableCF[] convert(
       Map<TableName, ? extends Collection<String>> tableCfs) {
     if (tableCfs == null) {
       return null;
     }
-    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
-    ZooKeeperProtos.TableCF.Builder tableCFBuilder =  ZooKeeperProtos.TableCF.newBuilder();
+    List<ReplicationProtos.TableCF> tableCFList = new ArrayList<>();
+    ReplicationProtos.TableCF.Builder tableCFBuilder =  ReplicationProtos.TableCF.newBuilder();
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       tableCFBuilder.clear();
       tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(entry.getKey()));
@@ -80,7 +80,7 @@ public final class ReplicationSerDeHelper {
       }
       tableCFList.add(tableCFBuilder.build());
     }
-    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+    return tableCFList.toArray(new ReplicationProtos.TableCF[tableCFList.size()]);
   }
 
   public static String convertToString(Map<TableName, ? extends Collection<String>> tableCfs) {
@@ -95,12 +95,12 @@ public final class ReplicationSerDeHelper {
    *  This is only for read TableCFs information from TableCF node.
    *  Input String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;ns3.table3.
    * */
-  public static ZooKeeperProtos.TableCF[] convert(String tableCFsConfig) {
+  public static ReplicationProtos.TableCF[] convert(String tableCFsConfig) {
     if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
       return null;
     }
-    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
-    ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder();
+    List<ReplicationProtos.TableCF> tableCFList = new ArrayList<>();
+    ReplicationProtos.TableCF.Builder tableCFBuilder = ReplicationProtos.TableCF.newBuilder();
 
     String[] tables = tableCFsConfig.split(";");
     for (String tab : tables) {
@@ -142,17 +142,17 @@ public final class ReplicationSerDeHelper {
       }
       tableCFList.add(tableCFBuilder.build());
     }
-    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+    return tableCFList.toArray(new ReplicationProtos.TableCF[tableCFList.size()]);
   }
 
   /**
    *  Convert TableCFs Object to String.
    *  Output String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;table3
    * */
-  public static String convert(ZooKeeperProtos.TableCF[] tableCFs) {
+  public static String convert(ReplicationProtos.TableCF[] tableCFs) {
     StringBuilder sb = new StringBuilder();
     for (int i = 0, n = tableCFs.length; i < n; i++) {
-      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      ReplicationProtos.TableCF tableCF = tableCFs[i];
       String namespace = tableCF.getTableName().getNamespace().toStringUtf8();
       if (!Strings.isEmpty(namespace)) {
         sb.append(namespace).append(".").
@@ -175,10 +175,10 @@ public final class ReplicationSerDeHelper {
   /**
    *  Get TableCF in TableCFs, if not exist, return null.
    * */
-  public static ZooKeeperProtos.TableCF getTableCF(ZooKeeperProtos.TableCF[] tableCFs,
+  public static ReplicationProtos.TableCF getTableCF(ReplicationProtos.TableCF[] tableCFs,
                                            String table) {
     for (int i = 0, n = tableCFs.length; i < n; i++) {
-      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      ReplicationProtos.TableCF tableCF = tableCFs[i];
       if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) {
         return tableCF;
       }
@@ -191,7 +191,7 @@ public final class ReplicationSerDeHelper {
    *  It is used for backward compatibility.
    *  Old format bytes have no PB_MAGIC Header
    * */
-  public static ZooKeeperProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
+  public static ReplicationProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
     if (bytes == null) {
       return null;
     }
@@ -202,20 +202,20 @@ public final class ReplicationSerDeHelper {
    *  Convert tableCFs string into Map.
    * */
   public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
-    ZooKeeperProtos.TableCF[] tableCFs = convert(tableCFsConfig);
+    ReplicationProtos.TableCF[] tableCFs = convert(tableCFsConfig);
     return convert2Map(tableCFs);
   }
 
   /**
    *  Convert tableCFs Object to Map.
    * */
-  public static Map<TableName, List<String>> convert2Map(ZooKeeperProtos.TableCF[] tableCFs) {
+  public static Map<TableName, List<String>> convert2Map(ReplicationProtos.TableCF[] tableCFs) {
     if (tableCFs == null || tableCFs.length == 0) {
       return null;
     }
     Map<TableName, List<String>> tableCFsMap = new HashMap<TableName, List<String>>();
     for (int i = 0, n = tableCFs.length; i < n; i++) {
-      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      ReplicationProtos.TableCF tableCF = tableCFs[i];
       List<String> families = new ArrayList<>();
       for (int j = 0, m = tableCF.getFamiliesCount(); j < m; j++) {
         families.add(tableCF.getFamilies(j).toStringUtf8());
@@ -239,9 +239,9 @@ public final class ReplicationSerDeHelper {
       throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
-      ZooKeeperProtos.ReplicationPeer.Builder builder =
-          ZooKeeperProtos.ReplicationPeer.newBuilder();
-      ZooKeeperProtos.ReplicationPeer peer;
+      ReplicationProtos.ReplicationPeer.Builder builder =
+          ReplicationProtos.ReplicationPeer.newBuilder();
+      ReplicationProtos.ReplicationPeer peer;
       try {
         ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
         peer = builder.build();
@@ -257,7 +257,7 @@ public final class ReplicationSerDeHelper {
     }
   }
 
-  public static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) {
+  public static ReplicationPeerConfig convert(ReplicationProtos.ReplicationPeer peer) {
     ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
     if (peer.hasClusterkey()) {
       peerConfig.setClusterKey(peer.getClusterkey());
@@ -275,7 +275,7 @@ public final class ReplicationSerDeHelper {
     }
 
     Map<TableName, ? extends Collection<String>> tableCFsMap = convert2Map(
-      peer.getTableCfsList().toArray(new ZooKeeperProtos.TableCF[peer.getTableCfsCount()]));
+      peer.getTableCfsList().toArray(new ReplicationProtos.TableCF[peer.getTableCfsCount()]));
     if (tableCFsMap != null) {
       peerConfig.setTableCFsMap(tableCFsMap);
     }
@@ -293,8 +293,8 @@ public final class ReplicationSerDeHelper {
     return peerConfig;
   }
 
-  public static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
-    ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
+  public static ReplicationProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
+    ReplicationProtos.ReplicationPeer.Builder builder = ReplicationProtos.ReplicationPeer.newBuilder();
     if (peerConfig.getClusterKey() != null) {
       builder.setClusterkey(peerConfig.getClusterKey());
     }
@@ -316,7 +316,7 @@ public final class ReplicationSerDeHelper {
           .build());
     }
 
-    ZooKeeperProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
+    ReplicationProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
     if (tableCFs != null) {
       for (int i = 0; i < tableCFs.length; i++) {
         builder.addTableCfs(tableCFs[i]);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index c58bd71..8b13f75 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -210,8 +210,8 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @throws DeserializationException
    */
   public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
-    ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
-    return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
+    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
+    return ReplicationProtos.ReplicationState.State.ENABLED == state;
   }
 
   /**
@@ -219,13 +219,13 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @return State parsed from the passed bytes.
    * @throws DeserializationException
    */
-  private static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
+  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
       throws DeserializationException {
     ProtobufUtil.expectPBMagicPrefix(bytes);
     int pblen = ProtobufUtil.lengthOfPBMagic();
-    ZooKeeperProtos.ReplicationState.Builder builder =
-        ZooKeeperProtos.ReplicationState.newBuilder();
-    ZooKeeperProtos.ReplicationState state;
+    ReplicationProtos.ReplicationState.Builder builder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    ReplicationProtos.ReplicationState state;
     try {
       ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
       state = builder.build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 9a617a7..a4b09c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@@ -160,13 +160,13 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   @Override
   public void enablePeer(String id) throws ReplicationException {
-    changePeerState(id, ZooKeeperProtos.ReplicationState.State.ENABLED);
+    changePeerState(id, ReplicationProtos.ReplicationState.State.ENABLED);
     LOG.info("peer " + id + " is enabled");
   }
 
   @Override
   public void disablePeer(String id) throws ReplicationException {
-    changePeerState(id, ZooKeeperProtos.ReplicationState.State.DISABLED);
+    changePeerState(id, ReplicationProtos.ReplicationState.State.DISABLED);
     LOG.info("peer " + id + " is disabled");
   }
 
@@ -462,7 +462,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
    * @param id
    * @param state
    */
-  private void changePeerState(String id, ZooKeeperProtos.ReplicationState.State state)
+  private void changePeerState(String id, ReplicationProtos.ReplicationState.State state)
       throws ReplicationException {
     try {
       if (!peerExists(id)) {
@@ -471,7 +471,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
       String peerStateZNode = getPeerStateNode(id);
       byte[] stateBytes =
-          (state == ZooKeeperProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
+          (state == ReplicationProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
               : DISABLED_ZNODE_BYTES;
       if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
         ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 2bfe757..c80822e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -23,13 +23,13 @@ import java.io.IOException;
 import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
 
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -66,9 +66,9 @@ public abstract class ReplicationStateZKBase {
 
   // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
-      toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
+      toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =
-      toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
+      toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
       "zookeeper.znode.replication.hfile.refs";
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
@@ -110,9 +110,9 @@ public abstract class ReplicationStateZKBase {
    *         use as content of a peer-state znode under a peer cluster id as in
    *         /hbase/replication/peers/PEER_ID/peer-state.
    */
-  protected static byte[] toByteArray(final ZooKeeperProtos.ReplicationState.State state) {
-    ZooKeeperProtos.ReplicationState msg =
-        ZooKeeperProtos.ReplicationState.newBuilder().setState(state).build();
+  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
+    ReplicationProtos.ReplicationState msg =
+        ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
     // There is no toByteArray on this pb Message?
     // 32 bytes is default which seems fair enough here.
     try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 2f6e1cf..98c96ec 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
@@ -51,7 +51,7 @@ 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.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
@@ -1860,8 +1860,8 @@ public class ZKUtil {
       }
       // parse the data of the above peer znode.
       try {
-        ZooKeeperProtos.ReplicationPeer.Builder builder =
-          ZooKeeperProtos.ReplicationPeer.newBuilder();
+        ReplicationProtos.ReplicationPeer.Builder builder =
+          ReplicationProtos.ReplicationPeer.newBuilder();
         ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
         String clusterKey = builder.getClusterkey();
         sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
@@ -1885,8 +1885,8 @@ public class ZKUtil {
       byte[] peerStateData;
       try {
         peerStateData = ZKUtil.getData(zkw, peerStateZnode);
-        ZooKeeperProtos.ReplicationState.Builder builder =
-            ZooKeeperProtos.ReplicationState.newBuilder();
+        ReplicationProtos.ReplicationState.Builder builder =
+            ReplicationProtos.ReplicationState.newBuilder();
         ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
         sb.append(builder.getState().name());
       } catch (IOException ipbe) {
@@ -2054,7 +2054,7 @@ public class ZKUtil {
    *         for use as content of an wal position in a replication queue.
    */
   public static byte[] positionToByteArray(final long position) {
-    byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
+    byte[] bytes = ReplicationProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
         .build().toByteArray();
     return ProtobufUtil.prependPBMagic(bytes);
   }
@@ -2070,9 +2070,9 @@ public class ZKUtil {
     }
     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
-      ZooKeeperProtos.ReplicationHLogPosition.Builder builder =
-          ZooKeeperProtos.ReplicationHLogPosition.newBuilder();
-      ZooKeeperProtos.ReplicationHLogPosition position;
+      ReplicationProtos.ReplicationHLogPosition.Builder builder =
+          ReplicationProtos.ReplicationHLogPosition.newBuilder();
+      ReplicationProtos.ReplicationHLogPosition position;
       try {
         ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
         position = builder.build();


[3/4] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by zg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
index 51d3fc9..e4fdfe8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
@@ -14,6 +14,3978 @@ public final class ReplicationProtos {
     registerAllExtensions(
         (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
+  public interface TableCFOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.TableCF)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getFamiliesList();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    int getFamiliesCount();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.TableCF}
+   */
+  public  static final class TableCF extends
+      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(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TableCF() {
+      families_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TableCF(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              families_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+
+    public static final int FAMILIES_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_;
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
+        getFamiliesList() {
+      return families_;
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public int getFamiliesCount() {
+      return families_.size();
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
+      return families_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (hasTableName()) {
+        if (!getTableName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    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());
+      }
+      for (int i = 0; i < families_.size(); i++) {
+        output.writeBytes(2, families_.get(i));
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        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 += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(families_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getFamiliesList().size();
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && getFamiliesList()
+          .equals(other.getFamiliesList());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (getFamiliesCount() > 0) {
+        hash = (37 * hash) + FAMILIES_FIELD_NUMBER;
+        hash = (53 * hash) + getFamiliesList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.TableCF}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.TableCF)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.families_ = families_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (!other.families_.isEmpty()) {
+          if (families_.isEmpty()) {
+            families_ = other.families_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensureFamiliesIsMutable();
+            families_.addAll(other.families_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
+      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_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              tableName_ != null &&
+              tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      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 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(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_ = java.util.Collections.emptyList();
+      private void ensureFamiliesIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>(families_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
+          getFamiliesList() {
+        return java.util.Collections.unmodifiableList(families_);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public int getFamiliesCount() {
+        return families_.size();
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
+        return families_.get(index);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder setFamilies(
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addFamilies(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addAllFamilies(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> values) {
+        ensureFamiliesIsMutable();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            values, families_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder clearFamilies() {
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.TableCF)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<TableCF>() {
+      public TableCF parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.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 org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface ReplicationPeerOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationPeer)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    boolean hasClusterkey();
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    java.lang.String getClusterkey();
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getClusterkeyBytes();
+
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    boolean hasReplicationEndpointImpl();
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    java.lang.String getReplicationEndpointImpl();
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getReplicationEndpointImplBytes();
+
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> 
+        getDataList();
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index);
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    int getDataCount();
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
+        getDataOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> 
+        getConfigurationList();
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index);
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    int getConfigurationCount();
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
+        getConfigurationOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF> 
+        getTableCfsList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getTableCfs(int index);
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    int getTableCfsCount();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getNamespacesList();
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    int getNamespacesCount();
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index);
+
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    boolean hasBandwidth();
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    long getBandwidth();
+  }
+  /**
+   * <pre>
+   **
+   * Used by replication. Holds a replication peer key.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.ReplicationPeer}
+   */
+  public  static final class ReplicationPeer extends
+      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(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private ReplicationPeer() {
+      clusterkey_ = "";
+      replicationEndpointImpl_ = "";
+      data_ = java.util.Collections.emptyList();
+      configuration_ = java.util.Collections.emptyList();
+      tableCfs_ = java.util.Collections.emptyList();
+      namespaces_ = java.util.Collections.emptyList();
+      bandwidth_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ReplicationPeer(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000001;
+              clusterkey_ = bs;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000002;
+              replicationEndpointImpl_ = bs;
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              data_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.PARSER, extensionRegistry));
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                configuration_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              configuration_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry));
+              break;
+            }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              tableCfs_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.PARSER, extensionRegistry));
+              break;
+            }
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                namespaces_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000020;
+              }
+              namespaces_.add(input.readBytes());
+              break;
+            }
+            case 56: {
+              bitField0_ |= 0x00000004;
+              bandwidth_ = input.readInt64();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          data_ = java.util.Collections.unmodifiableList(data_);
+        }
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          configuration_ = java.util.Collections.unmodifiableList(configuration_);
+        }
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+        }
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int CLUSTERKEY_FIELD_NUMBER = 1;
+    private volatile java.lang.Object clusterkey_;
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    public boolean hasClusterkey() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    public java.lang.String getClusterkey() {
+      java.lang.Object ref = clusterkey_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        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;
+        }
+        return s;
+      }
+    }
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getClusterkeyBytes() {
+      java.lang.Object ref = clusterkey_;
+      if (ref instanceof java.lang.String) {
+        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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int REPLICATIONENDPOINTIMPL_FIELD_NUMBER = 2;
+    private volatile java.lang.Object replicationEndpointImpl_;
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    public boolean hasReplicationEndpointImpl() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    public java.lang.String getReplicationEndpointImpl() {
+      java.lang.Object ref = replicationEndpointImpl_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        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;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getReplicationEndpointImplBytes() {
+      java.lang.Object ref = replicationEndpointImpl_;
+      if (ref instanceof java.lang.String) {
+        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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int DATA_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_;
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
+      return data_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
+        getDataOrBuilderList() {
+      return data_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public int getDataCount() {
+      return data_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
+      return data_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
+        int index) {
+      return data_.get(index);
+    }
+
+    public static final int CONFIGURATION_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> configuration_;
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> getConfigurationList() {
+      return configuration_;
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
+        getConfigurationOrBuilderList() {
+      return configuration_;
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public int getConfigurationCount() {
+      return configuration_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index) {
+      return configuration_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
+        int index) {
+      return configuration_.get(index);
+    }
+
+    public static final int TABLE_CFS_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF> tableCfs_;
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF> getTableCfsList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public int getTableCfsCount() {
+      return tableCfs_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getTableCfs(int index) {
+      return tableCfs_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index) {
+      return tableCfs_.get(index);
+    }
+
+    public static final int NAMESPACES_FIELD_NUMBER = 6;
+    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> namespaces_;
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
+        getNamespacesList() {
+      return namespaces_;
+    }
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    public int getNamespacesCount() {
+      return namespaces_.size();
+    }
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index) {
+      return namespaces_.get(index);
+    }
+
+    public static final int BANDWIDTH_FIELD_NUMBER = 7;
+    private long bandwidth_;
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    public boolean hasBandwidth() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    public long getBandwidth() {
+      return bandwidth_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasClusterkey()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getDataCount(); i++) {
+        if (!getData(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getConfigurationCount(); i++) {
+        if (!getConfiguration(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getTableCfsCount(); i++) {
+        if (!getTableCfs(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterkey_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        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));
+      }
+      for (int i = 0; i < configuration_.size(); i++) {
+        output.writeMessage(4, configuration_.get(i));
+      }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        output.writeMessage(5, tableCfs_.get(i));
+      }
+      for (int i = 0; i < namespaces_.size(); i++) {
+        output.writeBytes(6, namespaces_.get(i));
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt64(7, bandwidth_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterkey_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationEndpointImpl_);
+      }
+      for (int i = 0; i < data_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, data_.get(i));
+      }
+      for (int i = 0; i < configuration_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, configuration_.get(i));
+      }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        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 += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(namespaces_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getNamespacesList().size();
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeInt64Size(7, bandwidth_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer) obj;
+
+      boolean result = true;
+      result = result && (hasClusterkey() == other.hasClusterkey());
+      if (hasClusterkey()) {
+        result = result && getClusterkey()
+            .equals(other.getClusterkey());
+      }
+      result = result && (hasReplicationEndpointImpl() == other.hasReplicationEndpointImpl());
+      if (hasReplicationEndpointImpl()) {
+        result = result && getReplicationEndpointImpl()
+            .equals(other.getReplicationEndpointImpl());
+      }
+      result = result && getDataList()
+          .equals(other.getDataList());
+      result = result && getConfigurationList()
+          .equals(other.getConfigurationList());
+      result = result && getTableCfsList()
+          .equals(other.getTableCfsList());
+      result = result && getNamespacesList()
+          .equals(other.getNamespacesList());
+      result = result && (hasBandwidth() == other.hasBandwidth());
+      if (hasBandwidth()) {
+        result = result && (getBandwidth()
+            == other.getBandwidth());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasClusterkey()) {
+        hash = (37 * hash) + CLUSTERKEY_FIELD_NUMBER;
+        hash = (53 * hash) + getClusterkey().hashCode();
+      }
+      if (hasReplicationEndpointImpl()) {
+        hash = (37 * hash) + REPLICATIONENDPOINTIMPL_FIELD_NUMBER;
+        hash = (53 * hash) + getReplicationEndpointImpl().hashCode();
+      }
+      if (getDataCount() > 0) {
+        hash = (37 * hash) + DATA_FIELD_NUMBER;
+        hash = (53 * hash) + getDataList().hashCode();
+      }
+      if (getConfigurationCount() > 0) {
+        hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER;
+        hash = (53 * hash) + getConfigurationList().hashCode();
+      }
+      if (getTableCfsCount() > 0) {
+        hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER;
+        hash = (53 * hash) + getTableCfsList().hashCode();
+      }
+      if (getNamespacesCount() > 0) {
+        hash = (37 * hash) + NAMESPACES_FIELD_NUMBER;
+        hash = (53 * hash) + getNamespacesList().hashCode();
+      }
+      if (hasBandwidth()) {
+        hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getBandwidth());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     **
+     * Used by replication. Holds a replication peer key.
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.ReplicationPeer}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationPeer)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getDataFieldBuilder();
+          getConfigurationFieldBuilder();
+          getTableCfsFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        clusterkey_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        replicationEndpointImpl_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (dataBuilder_ == null) {
+          data_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          dataBuilder_.clear();
+        }
+        if (configurationBuilder_ == null) {
+          configuration_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000008);
+        } else {
+          configurationBuilder_.clear();
+        }
+        if (tableCfsBuilder_ == null) {
+          tableCfs_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+        } else {
+          tableCfsBuilder_.clear();
+        }
+        namespaces_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000020);
+        bandwidth_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clusterkey_ = clusterkey_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.replicationEndpointImpl_ = replicationEndpointImpl_;
+        if (dataBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            data_ = java.util.Collections.unmodifiableList(data_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.data_ = data_;
+        } else {
+          result.data_ = dataBuilder_.build();
+        }
+        if (configurationBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008)) {
+            configuration_ = java.util.Collections.unmodifiableList(configuration_);
+            bitField0_ = (bitField0_ & ~0x00000008);
+          }
+          result.configuration_ = configuration_;
+        } else {
+          result.configuration_ = configurationBuilder_.build();
+        }
+        if (tableCfsBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+            tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+            bitField0_ = (bitField0_ & ~0x00000010);
+          }
+          result.tableCfs_ = tableCfs_;
+        } else {
+          result.tableCfs_ = tableCfsBuilder_.build();
+        }
+        if (((bitField0_ & 0x00000020) == 0x00000020)) {
+          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
+          bitField0_ = (bitField0_ & ~0x00000020);
+        }
+        result.namespaces_ = namespaces_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.bandwidth_ = bandwidth_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance()) return this;
+        if (other.hasClusterkey()) {
+          bitField0_ |= 0x00000001;
+          clusterkey_ = other.clusterkey_;
+          onChanged();
+        }
+        if (other.hasReplicationEndpointImpl()) {
+          bitField0_ |= 0x00000002;
+          replicationEndpointImpl_ = other.replicationEndpointImpl_;
+          onChanged();
+        }
+        if (dataBuilder_ == null) {
+          if (!other.data_.isEmpty()) {
+            if (data_.isEmpty()) {
+              data_ = other.data_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureDataIsMutable();
+              data_.addAll(other.data_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.data_.isEmpty()) {
+            if (dataBuilder_.isEmpty()) {
+              dataBuilder_.dispose();
+              dataBuilder_ = null;
+              data_ = other.data_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              dataBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getDataFieldBuilder() : null;
+            } else {
+              dataBuilder_.addAllMessages(other.data_);
+            }
+          }
+        }
+        if (configurationBuilder_ == null) {
+          if (!other.configuration_.isEmpty()) {
+            if (configuration_.isEmpty()) {
+              configuration_ = other.configuration_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+            } else {
+              ensureConfigurationIsMutable();
+              configuration_.addAll(other.configuration_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.configuration_.isEmpty()) {
+            if (configurationBuilder_.isEmpty()) {
+              configurationBuilder_.dispose();
+              configurationBuilder_ = null;
+              configuration_ = other.configuration_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+              configurationBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getConfigurationFieldBuilder() : null;
+            } else {
+              configurationBuilder_.addAllMessages(other.configuration_);
+            }
+          }
+        }
+        if (tableCfsBuilder_ == null) {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfs_.isEmpty()) {
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+            } else {
+              ensureTableCfsIsMutable();
+              tableCfs_.addAll(other.tableCfs_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfsBuilder_.isEmpty()) {
+              tableCfsBuilder_.dispose();
+              tableCfsBuilder_ = null;
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+              tableCfsBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getTableCfsFieldBuilder() : null;
+            } else {
+              tableCfsBuilder_.addAllMessages(other.tableCfs_);
+            }
+          }
+        }
+        if (!other.namespaces_.isEmpty()) {
+          if (namespaces_.isEmpty()) {
+            namespaces_ = other.namespaces_;
+            bitField0_ = (bitField0_ & ~0x00000020);
+          } else {
+            ensureNamespacesIsMutable();
+            namespaces_.addAll(other.namespaces_);
+          }
+          onChanged();
+        }
+        if (other.hasBandwidth()) {
+          setBandwidth(other.getBandwidth());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasClusterkey()) {
+          return false;
+        }
+        for (int i = 0; i < getDataCount(); i++) {
+          if (!getData(i).isInitialized()) {
+            return false;
+          }
+        }
+        for (int i = 0; i < getConfigurationCount(); i++) {
+          if (!getConfiguration(i).isInitialized()) {
+            return false;
+          }
+        }
+        for (int i = 0; i < getTableCfsCount(); i++) {
+          if (!getTableCfs(i).isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.lang.Object clusterkey_ = "";
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public boolean hasClusterkey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public java.lang.String getClusterkey() {
+        java.lang.Object ref = clusterkey_;
+        if (!(ref instanceof java.lang.String)) {
+          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;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getClusterkeyBytes() {
+        java.lang.Object ref = clusterkey_;
+        if (ref instanceof String) {
+          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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public Builder setClusterkey(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        clusterkey_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public Builder clearClusterkey() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        clusterkey_ = getDefaultInstance().getClusterkey();
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public Builder setClusterkeyBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        clusterkey_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object replicationEndpointImpl_ = "";
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public boolean hasReplicationEndpointImpl() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public java.lang.String getReplicationEndpointImpl() {
+        java.lang.Object ref = replicationEndpointImpl_;
+        if (!(ref instanceof java.lang.String)) {
+          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;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getReplicationEndpointImplBytes() {
+        java.lang.Object ref = replicationEndpointImpl_;
+        if (ref instanceof String) {
+          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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public Builder setReplicationEndpointImpl(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        replicationEndpointImpl_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public Builder clearReplicationEndpointImpl() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        replicationEndpointImpl_ = getDefaultInstance().getReplicationEndpointImpl();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public Builder setReplicationEndpointImplBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        replicationEndpointImpl_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_ =
+        java.util.Collections.emptyList();
+      private void ensureDataIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>(data_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      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_;
+
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
+        if (dataBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(data_);
+        } else {
+          return dataBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public int getDataCount() {
+        if (dataBuilder_ == null) {
+          return data_.size();
+        } else {
+          return dataBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
+        if (dataBuilder_ == null) {
+          return data_.get(index);
+        } else {
+          return dataBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public Builder setData(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
+        if (dataBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureDataIsMutable();
+          data_.set(index, value);
+          onChanged();
+        } else {
+          dataBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public Builder setData(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
+        if (dataBuilder_ == null) {
+          ensureDataIsMutable();
+          data_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          dataBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public Builder addData(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
+        if (dataBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+      

<TRUNCATED>

[2/4] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by zg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
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 90ec659..de5827e 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
@@ -4120,3978 +4120,6 @@ public final class ZooKeeperProtos {
 
   }
 
-  public interface TableCFOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.TableCF)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    boolean hasTableName();
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
-
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getFamiliesList();
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    int getFamiliesCount();
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index);
-  }
-  /**
-   * Protobuf type {@code hbase.pb.TableCF}
-   */
-  public  static final class TableCF extends
-      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(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private TableCF() {
-      families_ = java.util.Collections.emptyList();
-    }
-
-    @java.lang.Override
-    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private TableCF(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      int mutable_bitField0_ = 0;
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = tableName_.toBuilder();
-              }
-              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(tableName_);
-                tableName_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000001;
-              break;
-            }
-            case 18: {
-              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000002;
-              }
-              families_.add(input.readBytes());
-              break;
-            }
-          }
-        }
-      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = java.util.Collections.unmodifiableList(families_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    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 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(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int TABLE_NAME_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    public boolean hasTableName() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
-      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-    }
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
-      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-    }
-
-    public static final int FAMILIES_FIELD_NUMBER = 2;
-    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_;
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
-        getFamiliesList() {
-      return families_;
-    }
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    public int getFamiliesCount() {
-      return families_.size();
-    }
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
-      return families_.get(index);
-    }
-
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (hasTableName()) {
-        if (!getTableName().isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    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());
-      }
-      for (int i = 0; i < families_.size(); i++) {
-        output.writeBytes(2, families_.get(i));
-      }
-      unknownFields.writeTo(output);
-    }
-
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        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 += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(families_.get(i));
-        }
-        size += dataSize;
-        size += 1 * getFamiliesList().size();
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF) obj;
-
-      boolean result = true;
-      result = result && (hasTableName() == other.hasTableName());
-      if (hasTableName()) {
-        result = result && getTableName()
-            .equals(other.getTableName());
-      }
-      result = result && getFamiliesList()
-          .equals(other.getFamiliesList());
-      result = result && unknownFields.equals(other.unknownFields);
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasTableName()) {
-        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getTableName().hashCode();
-      }
-      if (getFamiliesCount() > 0) {
-        hash = (37 * hash) + FAMILIES_FIELD_NUMBER;
-        hash = (53 * hash) + getFamiliesList().hashCode();
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString 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(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        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 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,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        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 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,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      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 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,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      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(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      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(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.TableCF}
-     */
-    public static final class Builder extends
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.TableCF)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder {
-      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 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(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-          getTableNameFieldBuilder();
-        }
-      }
-      public Builder clear() {
-        super.clear();
-        if (tableNameBuilder_ == null) {
-          tableName_ = null;
-        } else {
-          tableNameBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        families_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
-      }
-
-      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;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        if (tableNameBuilder_ == null) {
-          result.tableName_ = tableName_;
-        } else {
-          result.tableName_ = tableNameBuilder_.build();
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = java.util.Collections.unmodifiableList(families_);
-          bitField0_ = (bitField0_ & ~0x00000002);
-        }
-        result.families_ = families_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder clone() {
-        return (Builder) super.clone();
-      }
-      public Builder setField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.setField(field, value);
-      }
-      public Builder clearField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return (Builder) super.clearField(field);
-      }
-      public Builder clearOneof(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return (Builder) super.clearOneof(oneof);
-      }
-      public Builder setRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, Object value) {
-        return (Builder) super.setRepeatedField(field, index, value);
-      }
-      public Builder addRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.addRepeatedField(field, value);
-      }
-      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 {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance()) return this;
-        if (other.hasTableName()) {
-          mergeTableName(other.getTableName());
-        }
-        if (!other.families_.isEmpty()) {
-          if (families_.isEmpty()) {
-            families_ = other.families_;
-            bitField0_ = (bitField0_ & ~0x00000002);
-          } else {
-            ensureFamiliesIsMutable();
-            families_.addAll(other.families_);
-          }
-          onChanged();
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (hasTableName()) {
-          if (!getTableName().isInitialized()) {
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          org.apache.hadoop.hbase.shaded.com.google.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 (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 {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      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_;
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public boolean hasTableName() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
-        if (tableNameBuilder_ == null) {
-          return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-        } else {
-          return tableNameBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableNameBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          tableName_ = value;
-          onChanged();
-        } else {
-          tableNameBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder setTableName(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableNameBuilder_ == null) {
-          tableName_ = builderForValue.build();
-          onChanged();
-        } else {
-          tableNameBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableNameBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              tableName_ != null &&
-              tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
-            tableName_ =
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
-          } else {
-            tableName_ = value;
-          }
-          onChanged();
-        } else {
-          tableNameBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder clearTableName() {
-        if (tableNameBuilder_ == null) {
-          tableName_ = null;
-          onChanged();
-        } else {
-          tableNameBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getTableNameFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
-        if (tableNameBuilder_ != null) {
-          return tableNameBuilder_.getMessageOrBuilder();
-        } else {
-          return tableName_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-        }
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      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 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(),
-                  isClean());
-          tableName_ = null;
-        }
-        return tableNameBuilder_;
-      }
-
-      private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_ = java.util.Collections.emptyList();
-      private void ensureFamiliesIsMutable() {
-        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>(families_);
-          bitField0_ |= 0x00000002;
-         }
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
-          getFamiliesList() {
-        return java.util.Collections.unmodifiableList(families_);
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public int getFamiliesCount() {
-        return families_.size();
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
-        return families_.get(index);
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder setFamilies(
-          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureFamiliesIsMutable();
-        families_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder addFamilies(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureFamiliesIsMutable();
-        families_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder addAllFamilies(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> values) {
-        ensureFamiliesIsMutable();
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
-            values, families_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder clearFamilies() {
-        families_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000002);
-        onChanged();
-        return this;
-      }
-      public final Builder setUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.setUnknownFields(unknownFields);
-      }
-
-      public final Builder mergeUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.mergeUnknownFields(unknownFields);
-      }
-
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF)
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.TableCF)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF DEFAULT_INSTANCE;
-    static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF();
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstance() {
-      return DEFAULT_INSTANCE;
-    }
-
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<TableCF>() {
-      public TableCF parsePartialFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.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 org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> parser() {
-      return PARSER;
-    }
-
-    @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> getParserForType() {
-      return PARSER;
-    }
-
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() {
-      return DEFAULT_INSTANCE;
-    }
-
-  }
-
-  public interface ReplicationPeerOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationPeer)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    boolean hasClusterkey();
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    java.lang.String getClusterkey();
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getClusterkeyBytes();
-
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    boolean hasReplicationEndpointImpl();
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    java.lang.String getReplicationEndpointImpl();
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getReplicationEndpointImplBytes();
-
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> 
-        getDataList();
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index);
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    int getDataCount();
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
-        getDataOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
-        int index);
-
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> 
-        getConfigurationList();
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index);
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    int getConfigurationCount();
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
-        getConfigurationOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
-        int index);
-
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF> 
-        getTableCfsList();
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index);
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    int getTableCfsCount();
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
-        getTableCfsOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
-        int index);
-
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getNamespacesList();
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    int getNamespacesCount();
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index);
-
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    boolean hasBandwidth();
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    long getBandwidth();
-  }
-  /**
-   * <pre>
-   **
-   * Used by replication. Holds a replication peer key.
-   * </pre>
-   *
-   * Protobuf type {@code hbase.pb.ReplicationPeer}
-   */
-  public  static final class ReplicationPeer extends
-      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(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private ReplicationPeer() {
-      clusterkey_ = "";
-      replicationEndpointImpl_ = "";
-      data_ = java.util.Collections.emptyList();
-      configuration_ = java.util.Collections.emptyList();
-      tableCfs_ = java.util.Collections.emptyList();
-      namespaces_ = java.util.Collections.emptyList();
-      bandwidth_ = 0L;
-    }
-
-    @java.lang.Override
-    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private ReplicationPeer(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      int mutable_bitField0_ = 0;
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
-              bitField0_ |= 0x00000001;
-              clusterkey_ = bs;
-              break;
-            }
-            case 18: {
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
-              bitField0_ |= 0x00000002;
-              replicationEndpointImpl_ = bs;
-              break;
-            }
-            case 26: {
-              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>();
-                mutable_bitField0_ |= 0x00000004;
-              }
-              data_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.PARSER, extensionRegistry));
-              break;
-            }
-            case 34: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                configuration_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair>();
-                mutable_bitField0_ |= 0x00000008;
-              }
-              configuration_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry));
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              tableCfs_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                namespaces_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              namespaces_.add(input.readBytes());
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000004;
-              bandwidth_ = input.readInt64();
-              break;
-            }
-          }
-        }
-      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-          data_ = java.util.Collections.unmodifiableList(data_);
-        }
-        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-          configuration_ = java.util.Collections.unmodifiableList(configuration_);
-        }
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
-        }
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    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 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(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int CLUSTERKEY_FIELD_NUMBER = 1;
-    private volatile java.lang.Object clusterkey_;
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    public boolean hasClusterkey() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    public java.lang.String getClusterkey() {
-      java.lang.Object ref = clusterkey_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        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;
-        }
-        return s;
-      }
-    }
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getClusterkeyBytes() {
-      java.lang.Object ref = clusterkey_;
-      if (ref instanceof java.lang.String) {
-        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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    public static final int REPLICATIONENDPOINTIMPL_FIELD_NUMBER = 2;
-    private volatile java.lang.Object replicationEndpointImpl_;
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    public boolean hasReplicationEndpointImpl() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    public java.lang.String getReplicationEndpointImpl() {
-      java.lang.Object ref = replicationEndpointImpl_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        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;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getReplicationEndpointImplBytes() {
-      java.lang.Object ref = replicationEndpointImpl_;
-      if (ref instanceof java.lang.String) {
-        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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    public static final int DATA_FIELD_NUMBER = 3;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_;
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
-      return data_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
-        getDataOrBuilderList() {
-      return data_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public int getDataCount() {
-      return data_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
-      return data_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
-        int index) {
-      return data_.get(index);
-    }
-
-    public static final int CONFIGURATION_FIELD_NUMBER = 4;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> configuration_;
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> getConfigurationList() {
-      return configuration_;
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
-        getConfigurationOrBuilderList() {
-      return configuration_;
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public int getConfigurationCount() {
-      return configuration_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index) {
-      return configuration_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
-        int index) {
-      return configuration_.get(index);
-    }
-
-    public static final int TABLE_CFS_FIELD_NUMBER = 5;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF> tableCfs_;
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF> getTableCfsList() {
-      return tableCfs_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
-        getTableCfsOrBuilderList() {
-      return tableCfs_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public int getTableCfsCount() {
-      return tableCfs_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) {
-      return tableCfs_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
-        int index) {
-      return tableCfs_.get(index);
-    }
-
-    public static final int NAMESPACES_FIELD_NUMBER = 6;
-    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> namespaces_;
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
-        getNamespacesList() {
-      return namespaces_;
-    }
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    public int getNamespacesCount() {
-      return namespaces_.size();
-    }
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index) {
-      return namespaces_.get(index);
-    }
-
-    public static final int BANDWIDTH_FIELD_NUMBER = 7;
-    private long bandwidth_;
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    public boolean hasBandwidth() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    public long getBandwidth() {
-      return bandwidth_;
-    }
-
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (!hasClusterkey()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      for (int i = 0; i < getDataCount(); i++) {
-        if (!getData(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      for (int i = 0; i < getConfigurationCount(); i++) {
-        if (!getConfiguration(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      for (int i = 0; i < getTableCfsCount(); i++) {
-        if (!getTableCfs(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterkey_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        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));
-      }
-      for (int i = 0; i < configuration_.size(); i++) {
-        output.writeMessage(4, configuration_.get(i));
-      }
-      for (int i = 0; i < tableCfs_.size(); i++) {
-        output.writeMessage(5, tableCfs_.get(i));
-      }
-      for (int i = 0; i < namespaces_.size(); i++) {
-        output.writeBytes(6, namespaces_.get(i));
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(7, bandwidth_);
-      }
-      unknownFields.writeTo(output);
-    }
-
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterkey_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationEndpointImpl_);
-      }
-      for (int i = 0; i < data_.size(); i++) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(3, data_.get(i));
-      }
-      for (int i = 0; i < configuration_.size(); i++) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(4, configuration_.get(i));
-      }
-      for (int i = 0; i < tableCfs_.size(); i++) {
-        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 += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(namespaces_.get(i));
-        }
-        size += dataSize;
-        size += 1 * getNamespacesList().size();
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeInt64Size(7, bandwidth_);
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer) obj;
-
-      boolean result = true;
-      result = result && (hasClusterkey() == other.hasClusterkey());
-      if (hasClusterkey()) {
-        result = result && getClusterkey()
-            .equals(other.getClusterkey());
-      }
-      result = result && (hasReplicationEndpointImpl() == other.hasReplicationEndpointImpl());
-      if (hasReplicationEndpointImpl()) {
-        result = result && getReplicationEndpointImpl()
-            .equals(other.getReplicationEndpointImpl());
-      }
-      result = result && getDataList()
-          .equals(other.getDataList());
-      result = result && getConfigurationList()
-          .equals(other.getConfigurationList());
-      result = result && getTableCfsList()
-          .equals(other.getTableCfsList());
-      result = result && getNamespacesList()
-          .equals(other.getNamespacesList());
-      result = result && (hasBandwidth() == other.hasBandwidth());
-      if (hasBandwidth()) {
-        result = result && (getBandwidth()
-            == other.getBandwidth());
-      }
-      result = result && unknownFields.equals(other.unknownFields);
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasClusterkey()) {
-        hash = (37 * hash) + CLUSTERKEY_FIELD_NUMBER;
-        hash = (53 * hash) + getClusterkey().hashCode();
-      }
-      if (hasReplicationEndpointImpl()) {
-        hash = (37 * hash) + REPLICATIONENDPOINTIMPL_FIELD_NUMBER;
-        hash = (53 * hash) + getReplicationEndpointImpl().hashCode();
-      }
-      if (getDataCount() > 0) {
-        hash = (37 * hash) + DATA_FIELD_NUMBER;
-        hash = (53 * hash) + getDataList().hashCode();
-      }
-      if (getConfigurationCount() > 0) {
-        hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER;
-        hash = (53 * hash) + getConfigurationList().hashCode();
-      }
-      if (getTableCfsCount() > 0) {
-        hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER;
-        hash = (53 * hash) + getTableCfsList().hashCode();
-      }
-      if (getNamespacesCount() > 0) {
-        hash = (37 * hash) + NAMESPACES_FIELD_NUMBER;
-        hash = (53 * hash) + getNamespacesList().hashCode();
-      }
-      if (hasBandwidth()) {
-        hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getBandwidth());
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString 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(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        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 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,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        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 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,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      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 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,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      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(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      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(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * <pre>
-     **
-     * Used by replication. Holds a replication peer key.
-     * </pre>
-     *
-     * Protobuf type {@code hbase.pb.ReplicationPeer}
-     */
-    public static final class Builder extends
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationPeer)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeerOrBuilder {
-      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 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(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-          getDataFieldBuilder();
-          getConfigurationFieldBuilder();
-          getTableCfsFieldBuilder();
-        }
-      }
-      public Builder clear() {
-        super.clear();
-        clusterkey_ = "";
-        bitField0_ = (bitField0_ & ~0x00000001);
-        replicationEndpointImpl_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        if (dataBuilder_ == null) {
-          data_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000004);
-        } else {
-          dataBuilder_.clear();
-        }
-        if (configurationBuilder_ == null) {
-          configuration_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000008);
-        } else {
-          configurationBuilder_.clear();
-        }
-        if (tableCfsBuilder_ == null) {
-          tableCfs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        } else {
-          tableCfsBuilder_.clear();
-        }
-        namespaces_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000020);
-        bandwidth_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        return this;
-      }
-
-      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;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.clusterkey_ = clusterkey_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.replicationEndpointImpl_ = replicationEndpointImpl_;
-        if (dataBuilder_ == null) {
-          if (((bitField0_ & 0x00000004) == 0x00000004)) {
-            data_ = java.util.Collections.unmodifiableList(data_);
-            bitField0_ = (bitField0_ & ~0x00000004);
-          }
-          result.data_ = data_;
-        } else {
-          result.data_ = dataBuilder_.build();
-        }
-        if (configurationBuilder_ == null) {
-          if (((bitField0_ & 0x00000008) == 0x00000008)) {
-            configuration_ = java.util.Collections.unmodifiableList(configuration_);
-            bitField0_ = (bitField0_ & ~0x00000008);
-          }
-          result.configuration_ = configuration_;
-        } else {
-          result.configuration_ = configurationBuilder_.build();
-        }
-        if (tableCfsBuilder_ == null) {
-          if (((bitField0_ & 0x00000010) == 0x00000010)) {
-            tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
-            bitField0_ = (bitField0_ & ~0x00000010);
-          }
-          result.tableCfs_ = tableCfs_;
-        } else {
-          result.tableCfs_ = tableCfsBuilder_.build();
-        }
-        if (((bitField0_ & 0x00000020) == 0x00000020)) {
-          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
-          bitField0_ = (bitField0_ & ~0x00000020);
-        }
-        result.namespaces_ = namespaces_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.bandwidth_ = bandwidth_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder clone() {
-        return (Builder) super.clone();
-      }
-      public Builder setField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.setField(field, value);
-      }
-      public Builder clearField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return (Builder) super.clearField(field);
-      }
-      public Builder clearOneof(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return (Builder) super.clearOneof(oneof);
-      }
-      public Builder setRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, Object value) {
-        return (Builder) super.setRepeatedField(field, index, value);
-      }
-      public Builder addRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.addRepeatedField(field, value);
-      }
-      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 {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.getDefaultInstance()) return this;
-        if (other.hasClusterkey()) {
-          bitField0_ |= 0x00000001;
-          clusterkey_ = other.clusterkey_;
-          onChanged();
-        }
-        if (other.hasReplicationEndpointImpl()) {
-          bitField0_ |= 0x00000002;
-          replicationEndpointImpl_ = other.replicationEndpointImpl_;
-          onChanged();
-        }
-        if (dataBuilder_ == null) {
-          if (!other.data_.isEmpty()) {
-            if (data_.isEmpty()) {
-              data_ = other.data_;
-              bitField0_ = (bitField0_ & ~0x00000004);
-            } else {
-              ensureDataIsMutable();
-              data_.addAll(other.data_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.data_.isEmpty()) {
-            if (dataBuilder_.isEmpty()) {
-              dataBuilder_.dispose();
-              dataBuilder_ = null;
-              data_ = other.data_;
-              bitField0_ = (bitField0_ & ~0x00000004);
-              dataBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getDataFieldBuilder() : null;
-            } else {
-              dataBuilder_.addAllMessages(other.data_);
-            }
-          }
-        }
-        if (configurationBuilder_ == null) {
-          if (!other.configuration_.isEmpty()) {
-            if (configuration_.isEmpty()) {
-              configuration_ = other.configuration_;
-              bitField0_ = (bitField0_ & ~0x00000008);
-            } else {
-              ensureConfigurationIsMutable();
-              configuration_.addAll(other.configuration_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.configuration_.isEmpty()) {
-            if (configurationBuilder_.isEmpty()) {
-              configurationBuilder_.dispose();
-              configurationBuilder_ = null;
-              configuration_ = other.configuration_;
-              bitField0_ = (bitField0_ & ~0x00000008);
-              configurationBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getConfigurationFieldBuilder() : null;
-            } else {
-              configurationBuilder_.addAllMessages(other.configuration_);
-            }
-          }
-        }
-        if (tableCfsBuilder_ == null) {
-          if (!other.tableCfs_.isEmpty()) {
-            if (tableCfs_.isEmpty()) {
-              tableCfs_ = other.tableCfs_;
-              bitField0_ = (bitField0_ & ~0x00000010);
-            } else {
-              ensureTableCfsIsMutable();
-              tableCfs_.addAll(other.tableCfs_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.tableCfs_.isEmpty()) {
-            if (tableCfsBuilder_.isEmpty()) {
-              tableCfsBuilder_.dispose();
-              tableCfsBuilder_ = null;
-              tableCfs_ = other.tableCfs_;
-              bitField0_ = (bitField0_ & ~0x00000010);
-              tableCfsBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getTableCfsFieldBuilder() : null;
-            } else {
-              tableCfsBuilder_.addAllMessages(other.tableCfs_);
-            }
-          }
-        }
-        if (!other.namespaces_.isEmpty()) {
-          if (namespaces_.isEmpty()) {
-            namespaces_ = other.namespaces_;
-            bitField0_ = (bitField0_ & ~0x00000020);
-          } else {
-            ensureNamespacesIsMutable();
-            namespaces_.addAll(other.namespaces_);
-          }
-          onChanged();
-        }
-        if (other.hasBandwidth()) {
-          setBandwidth(other.getBandwidth());
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasClusterkey()) {
-          return false;
-        }
-        for (int i = 0; i < getDataCount(); i++) {
-          if (!getData(i).isInitialized()) {
-            return false;
-          }
-        }
-        for (int i = 0; i < getConfigurationCount(); i++) {
-          if (!getConfiguration(i).isInitialized()) {
-            return false;
-          }
-        }
-        for (int i = 0; i < getTableCfsCount(); i++) {
-          if (!getTableCfs(i).isInitialized()) {
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          org.apache.hadoop.hbase.shaded.com.google.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 (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 {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private java.lang.Object clusterkey_ = "";
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public boolean hasClusterkey() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public java.lang.String getClusterkey() {
-        java.lang.Object ref = clusterkey_;
-        if (!(ref instanceof java.lang.String)) {
-          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;
-          }
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-          getClusterkeyBytes() {
-        java.lang.Object ref = clusterkey_;
-        if (ref instanceof String) {
-          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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public Builder setClusterkey(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        clusterkey_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public Builder clearClusterkey() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        clusterkey_ = getDefaultInstance().getClusterkey();
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public Builder setClusterkeyBytes(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        clusterkey_ = value;
-        onChanged();
-        return this;
-      }
-
-      private java.lang.Object replicationEndpointImpl_ = "";
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public boolean hasReplicationEndpointImpl() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public java.lang.String getReplicationEndpointImpl() {
-        java.lang.Object ref = replicationEndpointImpl_;
-        if (!(ref instanceof java.lang.String)) {
-          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;
-          }
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-          getReplicationEndpointImplBytes() {
-        java.lang.Object ref = replicationEndpointImpl_;
-        if (ref instanceof String) {
-          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 (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public Builder setReplicationEndpointImpl(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        replicationEndpointImpl_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public Builder clearReplicationEndpointImpl() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        replicationEndpointImpl_ = getDefaultInstance().getReplicationEndpointImpl();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public Builder setReplicationEndpointImplBytes(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        replicationEndpointImpl_ = value;
-        onChanged();
-        return this;
-      }
-
-      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_ =
-        java.util.Collections.emptyList();
-      private void ensureDataIsMutable() {
-        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
-          data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>(data_);
-          bitField0_ |= 0x00000004;
-         }
-      }
-
-      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_;
-
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
-        if (dataBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(data_);
-        } else {
-          return dataBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public int getDataCount() {
-        if (dataBuilder_ == null) {
-          return data_.size();
-        } else {
-          return dataBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
-        if (dataBuilder_ == null) {
-          return data_.get(index);
-        } else {
-          return dataBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder setData(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
-        if (dataBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDataIsMutable();
-          data_.set(index, value);
-          onChanged();
-        } else {
-          dataBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder setData(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
-        if (dataBuilder_ == null) {
-          ensureDataIsMutable();
-          data_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          dataBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder addData(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
-        if (dataBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDataIsMutable();
-          data_.add(value);
-          onChanged();
-        } else {
-          dataBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder ad

<TRUNCATED>