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/10 01:06:29 UTC

[1/3] hbase git commit: HBASE-17337 list replication peers request should be routed through master

Repository: hbase
Updated Branches:
  refs/heads/master bd157ffe9 -> ac3b1c9aa


http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index f553e8f..7cd9921 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -893,6 +893,10 @@ service MasterService {
   rpc UpdateReplicationPeerConfig(UpdateReplicationPeerConfigRequest)
     returns(UpdateReplicationPeerConfigResponse);
 
+  /** Returns a list of replication peers */
+  rpc ListReplicationPeers(ListReplicationPeersRequest)
+    returns(ListReplicationPeersResponse);
+
   /** Returns a list of ServerNames marked as draining. */
   rpc listDrainingRegionServers(ListDrainingRegionServersRequest)
     returns(ListDrainingRegionServersResponse);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/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 c748514..4cb4436 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -58,6 +58,15 @@ message ReplicationState {
 }
 
 /**
+ * Used by replication. Description of the replication peer.
+ */
+message ReplicationPeerDescription {
+  required string id = 1;
+  required ReplicationState state = 2;
+  required ReplicationPeer config = 3;
+}
+
+/**
  * Used by replication. Holds the current position in an WAL file.
  */
 message ReplicationHLogPosition {
@@ -109,3 +118,11 @@ message UpdateReplicationPeerConfigRequest {
 
 message UpdateReplicationPeerConfigResponse {
 }
+
+message ListReplicationPeersRequest {
+  optional string regex = 1;
+}
+
+message ListReplicationPeersResponse {
+  repeated ReplicationPeerDescription peer_desc = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index c42c7b7..b0569e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -1951,4 +1951,24 @@ public interface MasterObserver extends Coprocessor {
       final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
       ReplicationPeerConfig peerConfig) throws IOException {
   }
+
+  /**
+   * Called before list replication peers.
+   * @param ctx
+   * @param regex The regular expression to match peer id
+   * @throws IOException on failure
+   */
+  default void preListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String regex) throws IOException {
+  }
+
+  /**
+   * Called after list replication peers.
+   * @param ctx
+   * @param regex The regular expression to match peer id
+   * @throws IOException on failure
+   */
+  default void postListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String regex) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index a87c38e..75da73f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -142,6 +142,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
@@ -3225,6 +3226,21 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
+  public List<ReplicationPeerDescription> listReplicationPeers(String regex)
+      throws ReplicationException, IOException {
+    if (cpHost != null) {
+      cpHost.preListReplicationPeers(regex);
+    }
+    LOG.info(getClientIdAuditPrefix() + " list replication peers, regex=" + regex);
+    Pattern pattern = regex == null ? null : Pattern.compile(regex);
+    List<ReplicationPeerDescription> peers = this.replicationManager.listReplicationPeers(pattern);
+    if (cpHost != null) {
+      cpHost.postListReplicationPeers(regex);
+    }
+    return peers;
+  }
+
+  @Override
   public void drainRegionServer(final ServerName server) {
     String parentZnode = getZooKeeper().znodePaths.drainingZNode;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 4e3987e..2b6ae39 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -1769,4 +1769,24 @@ public class MasterCoprocessorHost
       }
     });
   }
+
+  public void preListReplicationPeers(final String regex) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver observer, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        observer.preListReplicationPeers(ctx, regex);
+      }
+    });
+  }
+
+  public void postListReplicationPeers(final String regex) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver observer, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        observer.postListReplicationPeers(ctx, regex);
+      }
+    });
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 707cb39..732e678 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -95,6 +95,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Enabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
@@ -102,6 +104,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
@@ -1728,6 +1731,22 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
+      ListReplicationPeersRequest request) throws ServiceException {
+    ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();
+    try {
+      List<ReplicationPeerDescription> peers = master
+          .listReplicationPeers(request.hasRegex() ? request.getRegex() : null);
+      for (ReplicationPeerDescription peer : peers) {
+        response.addPeerDesc(ReplicationSerDeHelper.toProtoReplicationPeerDescription(peer));
+      }
+    } catch (ReplicationException | IOException e) {
+      throw new ServiceException(e);
+    }
+    return response.build();
+  }
+
+  @Override
   public ListDrainingRegionServersResponse listDrainingRegionServers(RpcController controller,
       ListDrainingRegionServersRequest request) throws ServiceException {
     ListDrainingRegionServersResponse.Builder response =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 7686fc8..060bdd8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 
 import com.google.protobuf.Service;
 
@@ -462,6 +463,14 @@ public interface MasterServices extends Server {
       throws ReplicationException, IOException;
 
   /**
+   * Return a list of replication peers.
+   * @param regex The regular expression to match peer id
+   * @return a list of replication peers description
+   */
+  List<ReplicationPeerDescription> listReplicationPeers(String regex) throws ReplicationException,
+      IOException;
+
+  /**
    * Mark a region server as draining to prevent additional regions from getting assigned to it.
    * @param server Region servers to drain.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
index f00730d..139a380 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
@@ -18,9 +18,12 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
@@ -30,6 +33,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
@@ -100,6 +104,20 @@ public class ReplicationManager {
     this.replicationPeers.updatePeerConfig(peerId, peerConfig);
   }
 
+  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern)
+      throws ReplicationException {
+    List<ReplicationPeerDescription> peers = new ArrayList<>();
+    List<String> peerIds = replicationPeers.getAllPeerIds();
+    for (String peerId : peerIds) {
+      if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
+        peers.add(new ReplicationPeerDescription(peerId, replicationPeers
+            .getStatusOfPeerFromBackingStore(peerId), replicationPeers
+            .getReplicationPeerConfig(peerId)));
+      }
+    }
+    return peers;
+  }
+
   /**
    * Set a namespace in the peer config means that all tables in this namespace
    * will be replicated to the peer cluster.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 0564ece..5011819 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -2733,4 +2733,10 @@ public class AccessController extends BaseMasterAndRegionObserver
       ReplicationPeerConfig peerConfig) throws IOException {
     requirePermission(getActiveUser(ctx), "updateReplicationPeerConfig", Action.ADMIN);
   }
-}
+
+  @Override
+  public void preListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String regex) throws IOException {
+    requirePermission(getActiveUser(ctx), "listReplicationPeers", Action.ADMIN);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 15d5150..435e20b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -207,7 +208,7 @@ public class TestReplicationAdmin {
     assertFalse(admin.getPeerState(ID_ONE));
     try {
       admin.getPeerState(ID_SECOND);
-    } catch (IllegalArgumentException iae) {
+    } catch (ReplicationPeerNotFoundException e) {
       // OK!
     }
     admin.removePeer(ID_ONE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 36dc2e2..10368b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -425,4 +426,10 @@ public class MockNoopMasterServices implements MasterServices, Server {
   public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
   }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers(String regex)
+      throws ReplicationException, IOException {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 7107669..5dcea53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -2961,4 +2961,19 @@ public class TestAccessController extends SecureTestUtil {
     verifyAllowed(action, SUPERUSER, USER_ADMIN);
     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
   }
+
+  @Test
+  public void testListReplicationPeers() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preListReplicationPeers(ObserverContext.createAndPrepare(CP_ENV, null),
+          "test");
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
index e902618..1d7c748 100644
--- a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
+++ b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
@@ -122,6 +122,7 @@ In case the table goes out of date, the unit tests which check for accuracy of p
 |        | disableReplicationPeer | superuser\|global(A)
 |        | getReplicationPeerConfig | superuser\|global(A)
 |        | updateReplicationPeerConfig | superuser\|global(A)
+|        | listReplicationPeers | superuser\|global(A)
 | Region | openRegion | superuser\|global(A)
 |        | closeRegion | superuser\|global(A)
 |        | flush | superuser\|global(A)\|global\(C)\|TableOwner\|table(A)\|table\(C)


[3/3] hbase git commit: HBASE-17337 list replication peers request should be routed through master

Posted by zg...@apache.org.
HBASE-17337 list replication peers request should be routed through master


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

Branch: refs/heads/master
Commit: ac3b1c9aa9d3ce5514afec0c2e0e34f9ac12d772
Parents: bd157ff
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Jan 10 08:57:26 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Tue Jan 10 08:57:26 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |   31 +
 .../hbase/client/ConnectionImplementation.java  |    8 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   31 +
 .../client/replication/ReplicationAdmin.java    |   27 +-
 .../replication/ReplicationSerDeHelper.java     |   25 +-
 .../replication/ReplicationPeerDescription.java |   59 +
 .../hbase/shaded/protobuf/ProtobufUtil.java     |    2 +
 .../hbase/shaded/protobuf/RequestConverter.java |    9 +
 .../shaded/protobuf/generated/MasterProtos.java |  142 +-
 .../protobuf/generated/ReplicationProtos.java   | 6002 ++++++++++++------
 .../src/main/protobuf/Master.proto              |    4 +
 .../src/main/protobuf/Replication.proto         |   17 +
 .../hbase/coprocessor/MasterObserver.java       |   20 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   16 +
 .../hbase/master/MasterCoprocessorHost.java     |   20 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   19 +
 .../hadoop/hbase/master/MasterServices.java     |    9 +
 .../master/replication/ReplicationManager.java  |   18 +
 .../hbase/security/access/AccessController.java |    8 +-
 .../replication/TestReplicationAdmin.java       |    3 +-
 .../hbase/master/MockNoopMasterServices.java    |    7 +
 .../security/access/TestAccessController.java   |   15 +
 .../asciidoc/_chapters/appendix_acl_matrix.adoc |    1 +
 23 files changed, 4614 insertions(+), 1879 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 2a1b782..4039fe3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Future;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -1880,6 +1882,35 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
+   * Return a list of replication peers.
+   * @return a list of replication peers description
+   * @throws IOException
+   */
+  default List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
+    return new ArrayList<>();
+  }
+
+  /**
+   * Return a list of replication peers.
+   * @param regex The regular expression to match peer id
+   * @return a list of replication peers description
+   * @throws IOException
+   */
+  default List<ReplicationPeerDescription> listReplicationPeers(String regex) throws IOException {
+    return new ArrayList<>();
+  }
+
+  /**
+   * Return a list of replication peers.
+   * @param pattern The compiled regular expression to match peer id
+   * @return a list of replication peers description
+   * @throws IOException
+   */
+  default List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException {
+    return new ArrayList<>();
+  }
+
+  /**
    * Mark a region server as draining to prevent additional regions from getting assigned to it.
    * @param servers List of region servers to drain.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index ea11c25..cfed9f6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -102,6 +102,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Enabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
@@ -1711,6 +1713,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           throws ServiceException {
         return stub.updateReplicationPeerConfig(controller, request);
       }
+
+      @Override
+      public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
+          ListReplicationPeersRequest request) throws ServiceException {
+        return stub.listReplicationPeers(controller, request);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 3c84929..db9cea5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -172,6 +173,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
@@ -3833,6 +3835,35 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
+    return listReplicationPeers((Pattern)null);
+  }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers(String regex) throws IOException {
+    return listReplicationPeers(Pattern.compile(regex));
+  }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern)
+      throws IOException {
+    return executeCallable(new MasterCallable<List<ReplicationPeerDescription>>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected List<ReplicationPeerDescription> rpcCall() throws Exception {
+        List<ReplicationProtos.ReplicationPeerDescription> peersList = master.listReplicationPeers(
+          getRpcController(), RequestConverter.buildListReplicationPeersRequest(pattern))
+            .getPeerDescList();
+        List<ReplicationPeerDescription> result = new ArrayList<>(peersList.size());
+        for (ReplicationProtos.ReplicationPeerDescription peer : peersList) {
+          result.add(ReplicationSerDeHelper.toReplicationPeerDescription(peer));
+        }
+        return result;
+      }
+    });
+  }
+
+  @Override
   public void drainRegionServers(List<ServerName> servers) throws IOException {
     final List<HBaseProtos.ServerName> pbServers = new ArrayList<HBaseProtos.ServerName>();
     for (ServerName server : servers) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index d0859a4..f9ca443 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -29,6 +29,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
 import java.util.Map.Entry;
 import java.util.Set;
 
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -52,6 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
@@ -238,13 +241,19 @@ public class ReplicationAdmin implements Closeable {
   /**
    * Get the number of slave clusters the local cluster has.
    * @return number of slave clusters
+   * @throws IOException
    */
-  public int getPeersCount() {
-    return this.replicationPeers.getAllPeerIds().size();
+  public int getPeersCount() throws IOException {
+    return this.admin.listReplicationPeers().size();
   }
 
-  public Map<String, ReplicationPeerConfig> listPeerConfigs() {
-    return this.replicationPeers.getAllPeerConfigs();
+  public Map<String, ReplicationPeerConfig> listPeerConfigs() throws IOException {
+    List<ReplicationPeerDescription> peers = this.admin.listReplicationPeers();
+    Map<String, ReplicationPeerConfig> result = new TreeMap<String, ReplicationPeerConfig>();
+    for (ReplicationPeerDescription peer : peers) {
+      result.put(peer.getPeerId(), peer.getPeerConfig());
+    }
+    return result;
   }
 
   public ReplicationPeerConfig getPeerConfig(String id) throws IOException {
@@ -402,8 +411,12 @@ public class ReplicationAdmin implements Closeable {
    * an IllegalArgumentException is thrown if it doesn't exist
    * @return true if replication is enabled to that peer, false if it isn't
    */
-  public boolean getPeerState(String id) throws ReplicationException {
-    return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
+  public boolean getPeerState(String id) throws ReplicationException, IOException {
+    List<ReplicationPeerDescription> peers = admin.listReplicationPeers(id);
+    if (peers.isEmpty() || !id.equals(peers.get(0).getPeerId())) {
+      throw new ReplicationPeerNotFoundException(id);
+    }
+    return peers.get(0).isEnabled();
   }
 
   @Override
@@ -577,7 +590,7 @@ public class ReplicationAdmin implements Closeable {
   }
 
   @VisibleForTesting
-  List<ReplicationPeer> listReplicationPeers() {
+  List<ReplicationPeer> listReplicationPeers() throws IOException {
     Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
     if (peers == null || peers.size() <= 0) {
       return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/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 9e04c9b..93eea17 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
@@ -31,6 +31,7 @@ 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.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
 
@@ -293,7 +294,7 @@ public final class ReplicationSerDeHelper {
     return peerConfig;
   }
 
-  public static ReplicationProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
+  public static ReplicationProtos.ReplicationPeer convert(ReplicationPeerConfig peerConfig) {
     ReplicationProtos.ReplicationPeer.Builder builder = ReplicationProtos.ReplicationPeer.newBuilder();
     if (peerConfig.getClusterKey() != null) {
       builder.setClusterkey(peerConfig.getClusterKey());
@@ -343,4 +344,26 @@ public final class ReplicationSerDeHelper {
     byte[] bytes = convert(peerConfig).toByteArray();
     return ProtobufUtil.prependPBMagic(bytes);
   }
+
+  public static ReplicationPeerDescription toReplicationPeerDescription(
+      ReplicationProtos.ReplicationPeerDescription desc) {
+    boolean enabled = ReplicationProtos.ReplicationState.State.ENABLED == desc.getState()
+        .getState();
+    ReplicationPeerConfig config = convert(desc.getConfig());
+    return new ReplicationPeerDescription(desc.getId(), enabled, config);
+  }
+
+  public static ReplicationProtos.ReplicationPeerDescription toProtoReplicationPeerDescription(
+      ReplicationPeerDescription desc) {
+    ReplicationProtos.ReplicationPeerDescription.Builder builder = ReplicationProtos.ReplicationPeerDescription
+        .newBuilder();
+    builder.setId(desc.getPeerId());
+    ReplicationProtos.ReplicationState.Builder stateBuilder = ReplicationProtos.ReplicationState
+        .newBuilder();
+    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED
+        : ReplicationProtos.ReplicationState.State.DISABLED);
+    builder.setState(stateBuilder.build());
+    builder.setConfig(convert(desc.getPeerConfig()));
+    return builder.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
new file mode 100644
index 0000000..577d13a
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ReplicationPeerDescription {
+
+  private final String id;
+  private final boolean enabled;
+  private final ReplicationPeerConfig config;
+
+  public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config) {
+    this.id = id;
+    this.enabled = enabled;
+    this.config = config;
+  }
+
+  public String getPeerId() {
+    return this.id;
+  }
+
+  public boolean isEnabled() {
+    return this.enabled;
+  }
+
+  public ReplicationPeerConfig getPeerConfig() {
+    return this.config;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder("id : ").append(id);
+    builder.append(", enabled : " + enabled);
+    builder.append(", config : " + config);
+    return builder.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 285e19a..585a5f8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.quotas.QuotaType;
 import org.apache.hadoop.hbase.quotas.ThrottleType;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
@@ -158,6 +159,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 4231a82..4acb525 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddRe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -1613,4 +1614,12 @@ public final class RequestConverter {
     builder.setPeerConfig(ReplicationSerDeHelper.convert(peerConfig));
     return builder.build();
   }
+
+  public static ListReplicationPeersRequest buildListReplicationPeersRequest(Pattern pattern) {
+    ListReplicationPeersRequest.Builder builder = ListReplicationPeersRequest.newBuilder();
+    if (pattern != null) {
+      builder.setRegex(pattern.toString());
+    }
+    return builder.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index d56c534..8f293f3 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -69799,6 +69799,18 @@ public final class MasterProtos {
 
       /**
        * <pre>
+       ** Returns a list of replication peers 
+       * </pre>
+       *
+       * <code>rpc ListReplicationPeers(.hbase.pb.ListReplicationPeersRequest) returns (.hbase.pb.ListReplicationPeersResponse);</code>
+       */
+      public abstract void listReplicationPeers(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse> done);
+
+      /**
+       * <pre>
        ** Returns a list of ServerNames marked as draining. 
        * </pre>
        *
@@ -70351,6 +70363,14 @@ public final class MasterProtos {
         }
 
         @java.lang.Override
+        public  void listReplicationPeers(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse> done) {
+          impl.listReplicationPeers(controller, request, done);
+        }
+
+        @java.lang.Override
         public  void listDrainingRegionServers(
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest request,
@@ -70525,10 +70545,12 @@ public final class MasterProtos {
             case 63:
               return impl.updateReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest)request);
             case 64:
-              return impl.listDrainingRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest)request);
+              return impl.listReplicationPeers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest)request);
             case 65:
-              return impl.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request);
+              return impl.listDrainingRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest)request);
             case 66:
+              return impl.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request);
+            case 67:
               return impl.removeDrainFromRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -70673,10 +70695,12 @@ public final class MasterProtos {
             case 63:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest.getDefaultInstance();
             case 64:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest.getDefaultInstance();
             case 65:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
             case 66:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+            case 67:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -70821,10 +70845,12 @@ public final class MasterProtos {
             case 63:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance();
             case 64:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance();
             case 65:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
             case 66:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+            case 67:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -71633,6 +71659,18 @@ public final class MasterProtos {
 
     /**
      * <pre>
+     ** Returns a list of replication peers 
+     * </pre>
+     *
+     * <code>rpc ListReplicationPeers(.hbase.pb.ListReplicationPeersRequest) returns (.hbase.pb.ListReplicationPeersResponse);</code>
+     */
+    public abstract void listReplicationPeers(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse> done);
+
+    /**
+     * <pre>
      ** Returns a list of ServerNames marked as draining. 
      * </pre>
      *
@@ -72010,16 +72048,21 @@ public final class MasterProtos {
               done));
           return;
         case 64:
+          this.listReplicationPeers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest)request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse>specializeCallback(
+              done));
+          return;
+        case 65:
           this.listDrainingRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse>specializeCallback(
               done));
           return;
-        case 65:
+        case 66:
           this.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse>specializeCallback(
               done));
           return;
-        case 66:
+        case 67:
           this.removeDrainFromRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse>specializeCallback(
               done));
@@ -72167,10 +72210,12 @@ public final class MasterProtos {
         case 63:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest.getDefaultInstance();
         case 64:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest.getDefaultInstance();
         case 65:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
         case 66:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+        case 67:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -72315,10 +72360,12 @@ public final class MasterProtos {
         case 63:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance();
         case 64:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance();
         case 65:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
         case 66:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+        case 67:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -73301,12 +73348,27 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance()));
       }
 
+      public  void listReplicationPeers(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(64),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.class,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance()));
+      }
+
       public  void listDrainingRegionServers(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(64),
+          getDescriptor().getMethods().get(65),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance(),
@@ -73321,7 +73383,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(65),
+          getDescriptor().getMethods().get(66),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance(),
@@ -73336,7 +73398,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(66),
+          getDescriptor().getMethods().get(67),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance(),
@@ -73673,6 +73735,11 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse listReplicationPeers(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse listDrainingRegionServers(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest request)
@@ -74464,12 +74531,24 @@ public final class MasterProtos {
       }
 
 
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse listReplicationPeers(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(64),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance());
+      }
+
+
       public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse listDrainingRegionServers(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(64),
+          getDescriptor().getMethods().get(65),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance());
@@ -74481,7 +74560,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(65),
+          getDescriptor().getMethods().get(66),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance());
@@ -74493,7 +74572,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(66),
+          getDescriptor().getMethods().get(67),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance());
@@ -75319,7 +75398,7 @@ public final class MasterProtos {
       "\022)\n\013server_name\030\001 \003(\0132\024.hbase.pb.ServerN" +
       "ame\"&\n$RemoveDrainFromRegionServersRespo",
       "nse*(\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022\t\n\005ME" +
-      "RGE\020\0012\2301\n\rMasterService\022e\n\024GetSchemaAlte" +
+      "RGE\020\0012\3771\n\rMasterService\022e\n\024GetSchemaAlte" +
       "rStatus\022%.hbase.pb.GetSchemaAlterStatusR" +
       "equest\032&.hbase.pb.GetSchemaAlterStatusRe" +
       "sponse\022b\n\023GetTableDescriptors\022$.hbase.pb" +
@@ -75468,17 +75547,20 @@ public final class MasterProtos {
       "\n\033UpdateReplicationPeerConfig\022,.hbase.pb" +
       ".UpdateReplicationPeerConfigRequest\032-.hb" +
       "ase.pb.UpdateReplicationPeerConfigRespon",
-      "se\022t\n\031listDrainingRegionServers\022*.hbase." +
-      "pb.ListDrainingRegionServersRequest\032+.hb" +
-      "ase.pb.ListDrainingRegionServersResponse" +
-      "\022_\n\022drainRegionServers\022#.hbase.pb.DrainR" +
-      "egionServersRequest\032$.hbase.pb.DrainRegi" +
-      "onServersResponse\022}\n\034removeDrainFromRegi" +
-      "onServers\022-.hbase.pb.RemoveDrainFromRegi" +
-      "onServersRequest\032..hbase.pb.RemoveDrainF" +
-      "romRegionServersResponseBI\n1org.apache.h" +
-      "adoop.hbase.shaded.protobuf.generatedB\014M",
-      "asterProtosH\001\210\001\001\240\001\001"
+      "se\022e\n\024ListReplicationPeers\022%.hbase.pb.Li" +
+      "stReplicationPeersRequest\032&.hbase.pb.Lis" +
+      "tReplicationPeersResponse\022t\n\031listDrainin" +
+      "gRegionServers\022*.hbase.pb.ListDrainingRe" +
+      "gionServersRequest\032+.hbase.pb.ListDraini" +
+      "ngRegionServersResponse\022_\n\022drainRegionSe" +
+      "rvers\022#.hbase.pb.DrainRegionServersReque" +
+      "st\032$.hbase.pb.DrainRegionServersResponse" +
+      "\022}\n\034removeDrainFromRegionServers\022-.hbase" +
+      ".pb.RemoveDrainFromRegionServersRequest\032",
+      "..hbase.pb.RemoveDrainFromRegionServersR" +
+      "esponseBI\n1org.apache.hadoop.hbase.shade" +
+      "d.protobuf.generatedB\014MasterProtosH\001\210\001\001\240" +
+      "\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {


[2/3] hbase git commit: HBASE-17337 list replication peers request should be routed through master

Posted by zg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ac3b1c9a/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 e4fdfe8..6a6d4bf 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
@@ -3502,37 +3502,68 @@ public final class ReplicationProtos {
 
   }
 
-  public interface ReplicationHLogPositionOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationHLogPosition)
+  public interface ReplicationPeerDescriptionOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationPeerDescription)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>required int64 position = 1;</code>
+     * <code>required string id = 1;</code>
      */
-    boolean hasPosition();
+    boolean hasId();
     /**
-     * <code>required int64 position = 1;</code>
+     * <code>required string id = 1;</code>
      */
-    long getPosition();
+    java.lang.String getId();
+    /**
+     * <code>required string id = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getIdBytes();
+
+    /**
+     * <code>required .hbase.pb.ReplicationState state = 2;</code>
+     */
+    boolean hasState();
+    /**
+     * <code>required .hbase.pb.ReplicationState state = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState getState();
+    /**
+     * <code>required .hbase.pb.ReplicationState state = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationStateOrBuilder getStateOrBuilder();
+
+    /**
+     * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+     */
+    boolean hasConfig();
+    /**
+     * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getConfig();
+    /**
+     * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder getConfigOrBuilder();
   }
   /**
    * <pre>
    **
-   * Used by replication. Holds the current position in an WAL file.
+   * Used by replication. Description of the replication peer.
    * </pre>
    *
-   * Protobuf type {@code hbase.pb.ReplicationHLogPosition}
+   * Protobuf type {@code hbase.pb.ReplicationPeerDescription}
    */
-  public  static final class ReplicationHLogPosition extends
+  public  static final class ReplicationPeerDescription extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationHLogPosition)
-      ReplicationHLogPositionOrBuilder {
-    // Use ReplicationHLogPosition.newBuilder() to construct.
-    private ReplicationHLogPosition(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationPeerDescription)
+      ReplicationPeerDescriptionOrBuilder {
+    // Use ReplicationPeerDescription.newBuilder() to construct.
+    private ReplicationPeerDescription(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private ReplicationHLogPosition() {
-      position_ = 0L;
+    private ReplicationPeerDescription() {
+      id_ = "";
     }
 
     @java.lang.Override
@@ -3540,7 +3571,7 @@ public final class ReplicationProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private ReplicationHLogPosition(
+    private ReplicationPeerDescription(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -3563,9 +3594,36 @@ public final class ReplicationProtos {
               }
               break;
             }
-            case 8: {
+            case 10: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
               bitField0_ |= 0x00000001;
-              position_ = input.readInt64();
+              id_ = bs;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = state_.toBuilder();
+              }
+              state_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(state_);
+                state_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = config_.toBuilder();
+              }
+              config_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(config_);
+                config_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
               break;
             }
           }
@@ -3582,30 +3640,99 @@ public final class ReplicationProtos {
     }
     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_ReplicationHLogPosition_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeerDescription_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_ReplicationHLogPosition_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeerDescription_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.Builder.class);
     }
 
     private int bitField0_;
-    public static final int POSITION_FIELD_NUMBER = 1;
-    private long position_;
+    public static final int ID_FIELD_NUMBER = 1;
+    private volatile java.lang.Object id_;
     /**
-     * <code>required int64 position = 1;</code>
+     * <code>required string id = 1;</code>
      */
-    public boolean hasPosition() {
+    public boolean hasId() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required int64 position = 1;</code>
+     * <code>required string id = 1;</code>
      */
-    public long getPosition() {
-      return position_;
+    public java.lang.String getId() {
+      java.lang.Object ref = id_;
+      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()) {
+          id_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string id = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getIdBytes() {
+      java.lang.Object ref = id_;
+      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);
+        id_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int STATE_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState state_;
+    /**
+     * <code>required .hbase.pb.ReplicationState state = 2;</code>
+     */
+    public boolean hasState() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .hbase.pb.ReplicationState state = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState getState() {
+      return state_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.getDefaultInstance() : state_;
+    }
+    /**
+     * <code>required .hbase.pb.ReplicationState state = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationStateOrBuilder getStateOrBuilder() {
+      return state_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.getDefaultInstance() : state_;
+    }
+
+    public static final int CONFIG_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer config_;
+    /**
+     * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+     */
+    public boolean hasConfig() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getConfig() {
+      return config_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance() : config_;
+    }
+    /**
+     * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder getConfigOrBuilder() {
+      return config_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance() : config_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -3614,7 +3741,23 @@ public final class ReplicationProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      if (!hasPosition()) {
+      if (!hasId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasState()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasConfig()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getState().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getConfig().isInitialized()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -3625,7 +3768,13 @@ public final class ReplicationProtos {
     public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeInt64(1, position_);
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, id_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, getState());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, getConfig());
       }
       unknownFields.writeTo(output);
     }
@@ -3636,8 +3785,15 @@ public final class ReplicationProtos {
 
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, id_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeInt64Size(1, position_);
+          .computeMessageSize(2, getState());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, getConfig());
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -3650,16 +3806,26 @@ public final class ReplicationProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription) obj;
 
       boolean result = true;
-      result = result && (hasPosition() == other.hasPosition());
-      if (hasPosition()) {
-        result = result && (getPosition()
-            == other.getPosition());
+      result = result && (hasId() == other.hasId());
+      if (hasId()) {
+        result = result && getId()
+            .equals(other.getId());
+      }
+      result = result && (hasState() == other.hasState());
+      if (hasState()) {
+        result = result && getState()
+            .equals(other.getState());
+      }
+      result = result && (hasConfig() == other.hasConfig());
+      if (hasConfig()) {
+        result = result && getConfig()
+            .equals(other.getConfig());
       }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
@@ -3672,68 +3838,75 @@ public final class ReplicationProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasPosition()) {
-        hash = (37 * hash) + POSITION_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getPosition());
+      if (hasId()) {
+        hash = (37 * hash) + ID_FIELD_NUMBER;
+        hash = (53 * hash) + getId().hashCode();
+      }
+      if (hasState()) {
+        hash = (37 * hash) + STATE_FIELD_NUMBER;
+        hash = (53 * hash) + getState().hashCode();
+      }
+      if (hasConfig()) {
+        hash = (37 * hash) + CONFIG_FIELD_NUMBER;
+        hash = (53 * hash) + getConfig().hashCode();
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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 {
@@ -3745,7 +3918,7 @@ public final class ReplicationProtos {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -3762,28 +3935,28 @@ public final class ReplicationProtos {
     /**
      * <pre>
      **
-     * Used by replication. Holds the current position in an WAL file.
+     * Used by replication. Description of the replication peer.
      * </pre>
      *
-     * Protobuf type {@code hbase.pb.ReplicationHLogPosition}
+     * Protobuf type {@code hbase.pb.ReplicationPeerDescription}
      */
     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.ReplicationHLogPosition)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPositionOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationPeerDescription)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescriptionOrBuilder {
       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_ReplicationHLogPosition_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeerDescription_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_ReplicationHLogPosition_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeerDescription_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -3796,40 +3969,70 @@ public final class ReplicationProtos {
       private void maybeForceBuilderInitialization() {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
+          getStateFieldBuilder();
+          getConfigFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        position_ = 0L;
+        id_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
+        if (stateBuilder_ == null) {
+          state_ = null;
+        } else {
+          stateBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (configBuilder_ == null) {
+          config_ = null;
+        } else {
+          configBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         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_ReplicationHLogPosition_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeerDescription_descriptor;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.getDefaultInstance();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition(this);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.position_ = position_;
+        result.id_ = id_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (stateBuilder_ == null) {
+          result.state_ = state_;
+        } else {
+          result.state_ = stateBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (configBuilder_ == null) {
+          result.config_ = config_;
+        } else {
+          result.config_ = configBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3862,18 +4065,26 @@ public final class ReplicationProtos {
         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.ReplicationHLogPosition) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition)other);
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.getDefaultInstance()) return this;
-        if (other.hasPosition()) {
-          setPosition(other.getPosition());
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription.getDefaultInstance()) return this;
+        if (other.hasId()) {
+          bitField0_ |= 0x00000001;
+          id_ = other.id_;
+          onChanged();
+        }
+        if (other.hasState()) {
+          mergeState(other.getState());
+        }
+        if (other.hasConfig()) {
+          mergeConfig(other.getConfig());
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -3881,7 +4092,19 @@ public final class ReplicationProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasPosition()) {
+        if (!hasId()) {
+          return false;
+        }
+        if (!hasState()) {
+          return false;
+        }
+        if (!hasConfig()) {
+          return false;
+        }
+        if (!getState().isInitialized()) {
+          return false;
+        }
+        if (!getConfig().isInitialized()) {
           return false;
         }
         return true;
@@ -3891,11 +4114,11 @@ public final class ReplicationProtos {
           org.apache.hadoop.hbase.shaded.com.google.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.ReplicationHLogPosition parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription 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.ReplicationHLogPosition) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -3906,32 +4129,796 @@ public final class ReplicationProtos {
       }
       private int bitField0_;
 
-      private long position_ ;
+      private java.lang.Object id_ = "";
       /**
-       * <code>required int64 position = 1;</code>
+       * <code>required string id = 1;</code>
        */
-      public boolean hasPosition() {
+      public boolean hasId() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required int64 position = 1;</code>
+       * <code>required string id = 1;</code>
        */
-      public long getPosition() {
-        return position_;
+      public java.lang.String getId() {
+        java.lang.Object ref = id_;
+        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()) {
+            id_ = s;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
       }
       /**
-       * <code>required int64 position = 1;</code>
+       * <code>required string id = 1;</code>
        */
-      public Builder setPosition(long value) {
-        bitField0_ |= 0x00000001;
-        position_ = value;
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getIdBytes() {
+        java.lang.Object ref = id_;
+        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);
+          id_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string id = 1;</code>
+       */
+      public Builder setId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        id_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required int64 position = 1;</code>
+       * <code>required string id = 1;</code>
        */
-      public Builder clearPosition() {
+      public Builder clearId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        id_ = getDefaultInstance().getId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string id = 1;</code>
+       */
+      public Builder setIdBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        id_ = value;
+        onChanged();
+        return this;
+      }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState state_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationStateOrBuilder> stateBuilder_;
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public boolean hasState() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState getState() {
+        if (stateBuilder_ == null) {
+          return state_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.getDefaultInstance() : state_;
+        } else {
+          return stateBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public Builder setState(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState value) {
+        if (stateBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          state_ = value;
+          onChanged();
+        } else {
+          stateBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public Builder setState(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.Builder builderForValue) {
+        if (stateBuilder_ == null) {
+          state_ = builderForValue.build();
+          onChanged();
+        } else {
+          stateBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public Builder mergeState(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState value) {
+        if (stateBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              state_ != null &&
+              state_ != org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.getDefaultInstance()) {
+            state_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.newBuilder(state_).mergeFrom(value).buildPartial();
+          } else {
+            state_ = value;
+          }
+          onChanged();
+        } else {
+          stateBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public Builder clearState() {
+        if (stateBuilder_ == null) {
+          state_ = null;
+          onChanged();
+        } else {
+          stateBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.Builder getStateBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getStateFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationStateOrBuilder getStateOrBuilder() {
+        if (stateBuilder_ != null) {
+          return stateBuilder_.getMessageOrBuilder();
+        } else {
+          return state_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.getDefaultInstance() : state_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationState state = 2;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationStateOrBuilder> 
+          getStateFieldBuilder() {
+        if (stateBuilder_ == null) {
+          stateBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationStateOrBuilder>(
+                  getState(),
+                  getParentForChildren(),
+                  isClean());
+          state_ = null;
+        }
+        return stateBuilder_;
+      }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer config_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder> configBuilder_;
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public boolean hasConfig() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getConfig() {
+        if (configBuilder_ == null) {
+          return config_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance() : config_;
+        } else {
+          return configBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public Builder setConfig(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer value) {
+        if (configBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          config_ = value;
+          onChanged();
+        } else {
+          configBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public Builder setConfig(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder builderForValue) {
+        if (configBuilder_ == null) {
+          config_ = builderForValue.build();
+          onChanged();
+        } else {
+          configBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public Builder mergeConfig(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer value) {
+        if (configBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              config_ != null &&
+              config_ != org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance()) {
+            config_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.newBuilder(config_).mergeFrom(value).buildPartial();
+          } else {
+            config_ = value;
+          }
+          onChanged();
+        } else {
+          configBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public Builder clearConfig() {
+        if (configBuilder_ == null) {
+          config_ = null;
+          onChanged();
+        } else {
+          configBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder getConfigBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getConfigFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder getConfigOrBuilder() {
+        if (configBuilder_ != null) {
+          return configBuilder_.getMessageOrBuilder();
+        } else {
+          return config_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance() : config_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.ReplicationPeer config = 3;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder> 
+          getConfigFieldBuilder() {
+        if (configBuilder_ == null) {
+          configBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder>(
+                  getConfig(),
+                  getParentForChildren(),
+                  isClean());
+          config_ = null;
+        }
+        return configBuilder_;
+      }
+      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.ReplicationPeerDescription)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ReplicationPeerDescription)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ReplicationPeerDescription>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<ReplicationPeerDescription>() {
+      public ReplicationPeerDescription 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 ReplicationPeerDescription(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ReplicationPeerDescription> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ReplicationPeerDescription> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerDescription getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface ReplicationHLogPositionOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationHLogPosition)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>required int64 position = 1;</code>
+     */
+    boolean hasPosition();
+    /**
+     * <code>required int64 position = 1;</code>
+     */
+    long getPosition();
+  }
+  /**
+   * <pre>
+   **
+   * Used by replication. Holds the current position in an WAL file.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.ReplicationHLogPosition}
+   */
+  public  static final class ReplicationHLogPosition extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationHLogPosition)
+      ReplicationHLogPositionOrBuilder {
+    // Use ReplicationHLogPosition.newBuilder() to construct.
+    private ReplicationHLogPosition(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private ReplicationHLogPosition() {
+      position_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ReplicationHLogPosition(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.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 8: {
+              bitField0_ |= 0x00000001;
+              position_ = 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 {
+        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_ReplicationHLogPosition_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_ReplicationHLogPosition_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int POSITION_FIELD_NUMBER = 1;
+    private long position_;
+    /**
+     * <code>required int64 position = 1;</code>
+     */
+    public boolean hasPosition() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required int64 position = 1;</code>
+     */
+    public long getPosition() {
+      return position_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasPosition()) {
+        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.writeInt64(1, position_);
+      }
+      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
+          .computeInt64Size(1, position_);
+      }
+      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.ReplicationHLogPosition)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition) obj;
+
+      boolean result = true;
+      result = result && (hasPosition() == other.hasPosition());
+      if (hasPosition()) {
+        result = result && (getPosition()
+            == other.getPosition());
+      }
+      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 (hasPosition()) {
+        hash = (37 * hash) + POSITION_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getPosition());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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.ReplicationHLogPosition 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 the current position in an WAL file.
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.ReplicationHLogPosition}
+     */
+    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.ReplicationHLogPosition)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPositionOrBuilder {
+      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_ReplicationHLogPosition_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_ReplicationHLogPosition_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.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) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        position_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        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_ReplicationHLogPosition_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.position_ = position_;
+        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.ReplicationHLogPosition) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition.getDefaultInstance()) return this;
+        if (other.hasPosition()) {
+          setPosition(other.getPosition());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasPosition()) {
+          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.ReplicationHLogPosition 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.ReplicationHLogPosition) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private long position_ ;
+      /**
+       * <code>required int64 position = 1;</code>
+       */
+      public boolean hasPosition() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required int64 position = 1;</code>
+       */
+      public long getPosition() {
+        return position_;
+      }
+      /**
+       * <code>required int64 position = 1;</code>
+       */
+      public Builder setPosition(long value) {
+        bitField0_ |= 0x00000001;
+        position_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 position = 1;</code>
+       */
+      public Builder clearPosition() {
         bitField0_ = (bitField0_ & ~0x00000001);
         position_ = 0L;
         onChanged();
@@ -3948,46 +4935,1183 @@ public final class ReplicationProtos {
       }
 
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicationHLogPosition)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicationHLogPosition)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ReplicationHLogPosition)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ReplicationHLogPosition>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<ReplicationHLogPosition>() {
+      public ReplicationHLogPosition 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 ReplicationHLogPosition(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ReplicationHLogPosition> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ReplicationHLogPosition> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationHLogPosition getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface AddReplicationPeerRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.AddReplicationPeerRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>required string peer_id = 1;</code>
+     */
+    boolean hasPeerId();
+    /**
+     * <code>required string peer_id = 1;</code>
+     */
+    java.lang.String getPeerId();
+    /**
+     * <code>required string peer_id = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getPeerIdBytes();
+
+    /**
+     * <code>required .hbase.pb.ReplicationPeer peer_config = 2;</code>
+     */
+    boolean hasPeerConfig();
+    /**
+     * <code>required .hbase.pb.ReplicationPeer peer_config = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getPeerConfig();
+    /**
+     * <code>required .hbase.pb.ReplicationPeer peer_config = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder getPeerConfigOrBuilder();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.AddReplicationPeerRequest}
+   */
+  public  static final class AddReplicationPeerRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.AddReplicationPeerRequest)
+      AddReplicationPeerRequestOrBuilder {
+    // Use AddReplicationPeerRequest.newBuilder() to construct.
+    private AddReplicationPeerRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private AddReplicationPeerRequest() {
+      peerId_ = "";
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private AddReplicationPeerRequest(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.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;
+              peerId_ = bs;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = peerConfig_.toBuilder();
+              }
+              peerConfig_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(peerConfig_);
+                peerConfig_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              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 {
+        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_AddReplicationPeerRequest_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_AddReplicationPeerRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int PEER_ID_FIELD_NUMBER = 1;
+    private volatile java.lang.Object peerId_;
+    /**
+     * <code>required string peer_id = 1;</code>
+     */
+    public boolean hasPeerId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string peer_id = 1;</code>
+     */
+    public java.lang.String getPeerId() {
+      java.lang.Object ref = peerId_;
+      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()) {
+          peerId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string peer_id = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getPeerIdBytes() {
+      java.lang.Object ref = peerId_;
+      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);
+        peerId_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int PEER_CONFIG_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer peerConfig_;
+    /**
+     * <code>required .hbase.pb.ReplicationPeer peer_config = 2;</code>
+     */
+    public boolean hasPeerConfig() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .hbase.pb.ReplicationPeer peer_config = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getPeerConfig() {
+      return peerConfig_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance() : peerConfig_;
+    }
+    /**
+     * <code>required .hbase.pb.ReplicationPeer peer_config = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder getPeerConfigOrBuilder() {
+      return peerConfig_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance() : peerConfig_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasPeerId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasPeerConfig()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getPeerConfig().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, peerId_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, getPeerConfig());
+      }
+      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, peerId_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getPeerConfig());
+      }
+      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.AddReplicationPeerRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest) obj;
+
+      boolean result = true;
+      result = result && (hasPeerId() == other.hasPeerId());
+      if (hasPeerId()) {
+        result = result && getPeerId()
+            .equals(other.getPeerId());
+      }
+      result = result && (hasPeerConfig() == other.hasPeerConfig());
+      if (hasPeerConfig()) {
+        result = result && getPeerConfig()
+            .equals(other.getPeerConfig());
+      }
+      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 (hasPeerId()) {
+        hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
+        hash = (53 * hash) + getPeerId().hashCode();
+      }
+      if (hasPeerConfig()) {
+        hash = (37 * hash) + PEER_CONFIG_FIELD_NUMBER;
+        hash = (53 * hash) + getPeerConfig().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest 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.AddReplicationPeerRequest}
+     */
+    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.AddReplicationPeerRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequestOrBuilder {
+      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_AddReplicationPeerRequest_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_AddReplicationPeerRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.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) {
+          getPeerConfigFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        peerId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (peerConfigBuilder_ == null) {
+          peerConfig_ = null;
+        } else {
+          peerConfigBuilder_.clear();
+        }
+        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_AddReplicationPeerRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.Repli

<TRUNCATED>