You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2021/01/24 15:37:26 UTC

[hbase] branch master updated: HBASE-25522 Remove deprecated methods in ReplicationPeerConfig (#2898)

This is an automated email from the ASF dual-hosted git repository.

vjasani pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 0b0cdca  HBASE-25522 Remove deprecated methods in ReplicationPeerConfig (#2898)
0b0cdca is described below

commit 0b0cdcac5352b12b907c32eb0fec6a7c65fe4904
Author: Baiqiang Zhao <zb...@gmail.com>
AuthorDate: Sun Jan 24 23:37:03 2021 +0800

    HBASE-25522 Remove deprecated methods in ReplicationPeerConfig (#2898)
    
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../hbase/replication/ReplicationPeerConfig.java   |  97 --------------------
 .../hbase/client/TestAsyncReplicationAdminApi.java | 102 ++++++++++-----------
 .../TestAsyncReplicationAdminApiWithClusters.java  |  27 +++---
 .../hbase/client/TestReplicaWithCluster.java       |   4 +-
 .../hbase/replication/TestMasterReplication.java   |  10 +-
 .../replication/TestMultiSlaveReplication.java     |  11 +--
 .../hbase/replication/TestReplicationEndpoint.java |  60 +++++++-----
 .../hbase/replication/TestReplicationWithTags.java |   4 +-
 .../TestGlobalReplicationThrottler.java            |   4 +-
 .../replication/regionserver/TestReplicator.java   |   9 +-
 .../security/access/TestAccessController.java      |   3 +-
 ...tVisibilityLabelReplicationWithExpAsString.java |   5 +-
 .../TestVisibilityLabelsReplication.java           |   4 +-
 13 files changed, 125 insertions(+), 215 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index bb3ff04..5ca5cef 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -79,41 +79,6 @@ public class ReplicationPeerConfig {
     return Collections.unmodifiableMap(newTableCFsMap);
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder} to create new ReplicationPeerConfig.
-   */
-  @Deprecated
-  public ReplicationPeerConfig() {
-    this.peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    this.configuration = new HashMap<>(0);
-    this.serial = false;
-  }
-
-  /**
-   * Set the clusterKey which is the concatenation of the slave cluster's:
-   * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setClusterKey(String)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setClusterKey(String clusterKey) {
-    this.clusterKey = clusterKey;
-    return this;
-  }
-
-  /**
-   * Sets the ReplicationEndpoint plugin class for this peer.
-   * @param replicationEndpointImpl a class implementing ReplicationEndpoint
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setReplicationEndpointImpl(String)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setReplicationEndpointImpl(String replicationEndpointImpl) {
-    this.replicationEndpointImpl = replicationEndpointImpl;
-    return this;
-  }
-
   public String getClusterKey() {
     return clusterKey;
   }
@@ -134,88 +99,26 @@ public class ReplicationPeerConfig {
     return (Map<TableName, List<String>>) tableCFsMap;
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setTableCFsMap(Map)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setTableCFsMap(Map<TableName,
-                                              ? extends Collection<String>> tableCFsMap) {
-    this.tableCFsMap = tableCFsMap;
-    return this;
-  }
-
   public Set<String> getNamespaces() {
     return this.namespaces;
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setNamespaces(Set)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setNamespaces(Set<String> namespaces) {
-    this.namespaces = namespaces;
-    return this;
-  }
-
   public long getBandwidth() {
     return this.bandwidth;
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setBandwidth(long)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setBandwidth(long bandwidth) {
-    this.bandwidth = bandwidth;
-    return this;
-  }
-
   public boolean replicateAllUserTables() {
     return this.replicateAllUserTables;
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setReplicateAllUserTables(boolean)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setReplicateAllUserTables(boolean replicateAllUserTables) {
-    this.replicateAllUserTables = replicateAllUserTables;
-    return this;
-  }
-
   public Map<TableName, List<String>> getExcludeTableCFsMap() {
     return (Map<TableName, List<String>>) excludeTableCFsMap;
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setExcludeTableCFsMap(Map)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setExcludeTableCFsMap(Map<TableName,
-                                              ? extends Collection<String>> tableCFsMap) {
-    this.excludeTableCFsMap = tableCFsMap;
-    return this;
-  }
-
   public Set<String> getExcludeNamespaces() {
     return this.excludeNamespaces;
   }
 
-  /**
-   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
-   *             {@link ReplicationPeerConfigBuilder#setExcludeNamespaces(Set)} instead.
-   */
-  @Deprecated
-  public ReplicationPeerConfig setExcludeNamespaces(Set<String> namespaces) {
-    this.excludeNamespaces = namespaces;
-    return this;
-  }
-
   public String getRemoteWALDir() {
     return this.remoteWALDir;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
index 74b5c2f..479fe6b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
@@ -108,10 +109,8 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testAddRemovePeer() throws Exception {
-    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
-    rpc1.setClusterKey(KEY_ONE);
-    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
-    rpc2.setClusterKey(KEY_TWO);
+    ReplicationPeerConfig rpc1 = ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build();
+    ReplicationPeerConfig rpc2 = ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build();
     // Add a valid peer
     admin.addReplicationPeer(ID_ONE, rpc1).join();
     // try adding the same (fails)
@@ -142,10 +141,11 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testPeerConfig() throws Exception {
-    ReplicationPeerConfig config = new ReplicationPeerConfig();
-    config.setClusterKey(KEY_ONE);
-    config.getConfiguration().put("key1", "value1");
-    config.getConfiguration().put("key2", "value2");
+    ReplicationPeerConfig config = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(KEY_ONE)
+      .putConfiguration("key1", "value1")
+      .putConfiguration("key2", "value2")
+      .build();
     admin.addReplicationPeer(ID_ONE, config).join();
 
     List<ReplicationPeerDescription> peers = admin.listReplicationPeers().get();
@@ -160,8 +160,7 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testEnableDisablePeer() throws Exception {
-    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
-    rpc1.setClusterKey(KEY_ONE);
+    ReplicationPeerConfig rpc1 = ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build();
     admin.addReplicationPeer(ID_ONE, rpc1).join();
     List<ReplicationPeerDescription> peers = admin.listReplicationPeers().get();
     assertEquals(1, peers.size());
@@ -176,8 +175,8 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testAppendPeerTableCFs() throws Exception {
-    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
-    rpc1.setClusterKey(KEY_ONE);
+    ReplicationPeerConfigBuilder rpcBuilder =
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE);
     final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "t1");
     final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "t2");
     final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "t3");
@@ -186,9 +185,9 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     final TableName tableName6 = TableName.valueOf(tableName.getNameAsString() + "t6");
 
     // Add a valid peer
-    admin.addReplicationPeer(ID_ONE, rpc1).join();
-    rpc1.setReplicateAllUserTables(false);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc1).join();
+    admin.addReplicationPeer(ID_ONE, rpcBuilder.build()).join();
+    rpcBuilder.setReplicateAllUserTables(false);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
 
     Map<TableName, List<String>> tableCFs = new HashMap<>();
 
@@ -280,16 +279,16 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testRemovePeerTableCFs() throws Exception {
-    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
-    rpc1.setClusterKey(KEY_ONE);
+    ReplicationPeerConfigBuilder rpcBuilder =
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE);
     final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "t1");
     final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "t2");
     final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "t3");
     final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "t4");
     // Add a valid peer
-    admin.addReplicationPeer(ID_ONE, rpc1).join();
-    rpc1.setReplicateAllUserTables(false);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc1).join();
+    admin.addReplicationPeer(ID_ONE, rpcBuilder.build()).join();
+    rpcBuilder.setReplicateAllUserTables(false);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
 
     Map<TableName, List<String>> tableCFs = new HashMap<>();
     try {
@@ -369,30 +368,28 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     String ns1 = "ns1";
     String ns2 = "ns2";
 
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(KEY_ONE);
-    admin.addReplicationPeer(ID_ONE, rpc).join();
-    rpc.setReplicateAllUserTables(false);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    ReplicationPeerConfigBuilder rpcBuilder =
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE);
+    admin.addReplicationPeer(ID_ONE, rpcBuilder.build()).join();
+    rpcBuilder.setReplicateAllUserTables(false);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
 
     // add ns1 and ns2 to peer config
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
     Set<String> namespaces = new HashSet<>();
     namespaces.add(ns1);
     namespaces.add(ns2);
-    rpc.setNamespaces(namespaces);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    rpcBuilder.setNamespaces(namespaces);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
     namespaces = admin.getReplicationPeerConfig(ID_ONE).get().getNamespaces();
     assertEquals(2, namespaces.size());
     assertTrue(namespaces.contains(ns1));
     assertTrue(namespaces.contains(ns2));
 
     // update peer config only contains ns1
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
     namespaces = new HashSet<>();
     namespaces.add(ns1);
-    rpc.setNamespaces(namespaces);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    rpcBuilder.setNamespaces(namespaces);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
     namespaces = admin.getReplicationPeerConfig(ID_ONE).get().getNamespaces();
     assertEquals(1, namespaces.size());
     assertTrue(namespaces.contains(ns1));
@@ -407,40 +404,36 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     final TableName tableName1 = TableName.valueOf(ns1 + ":" + tableName.getNameAsString() + "1");
     final TableName tableName2 = TableName.valueOf(ns2 + ":" + tableName.getNameAsString() + "2");
 
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(KEY_ONE);
-    admin.addReplicationPeer(ID_ONE, rpc).join();
-    rpc.setReplicateAllUserTables(false);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    ReplicationPeerConfigBuilder rpcBuilder =
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE);
+    admin.addReplicationPeer(ID_ONE, rpcBuilder.build()).join();
+    rpcBuilder.setReplicateAllUserTables(false);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
 
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
     Set<String> namespaces = new HashSet<String>();
     namespaces.add(ns1);
-    rpc.setNamespaces(namespaces);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).get();
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    rpcBuilder.setNamespaces(namespaces);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).get();
     Map<TableName, List<String>> tableCfs = new HashMap<>();
     tableCfs.put(tableName1, new ArrayList<>());
-    rpc.setTableCFsMap(tableCfs);
+    rpcBuilder.setTableCFsMap(tableCfs);
     try {
-      admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+      admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
       fail(
         "Test case should fail, because table " + tableName1 + " conflict with namespace " + ns1);
     } catch (CompletionException e) {
       // OK
     }
 
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
     tableCfs.clear();
     tableCfs.put(tableName2, new ArrayList<>());
-    rpc.setTableCFsMap(tableCfs);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).get();
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    rpcBuilder.setTableCFsMap(tableCfs);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).get();
     namespaces.clear();
     namespaces.add(ns2);
-    rpc.setNamespaces(namespaces);
+    rpcBuilder.setNamespaces(namespaces);
     try {
-      admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+      admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
       fail(
         "Test case should fail, because namespace " + ns2 + " conflict with table " + tableName2);
     } catch (CompletionException e) {
@@ -452,15 +445,14 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   @Test
   public void testPeerBandwidth() throws Exception {
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(KEY_ONE);
+    ReplicationPeerConfigBuilder rpcBuilder =
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE);
 
-    admin.addReplicationPeer(ID_ONE, rpc).join();
-    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
-    assertEquals(0, rpc.getBandwidth());
+    admin.addReplicationPeer(ID_ONE, rpcBuilder.build()).join();;
+    assertEquals(0, admin.getReplicationPeerConfig(ID_ONE).get().getBandwidth());
 
-    rpc.setBandwidth(2097152);
-    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    rpcBuilder.setBandwidth(2097152);
+    admin.updateReplicationPeerConfig(ID_ONE, rpcBuilder.build()).join();
     assertEquals(2097152, admin.getReplicationPeerConfig(ID_ONE).join().getBandwidth());
 
     admin.removeReplicationPeer(ID_ONE).join();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
index 1fb9df6..c959963 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java
@@ -24,8 +24,8 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CompletionException;
 import java.util.concurrent.ForkJoinPool;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -86,8 +87,8 @@ public class TestAsyncReplicationAdminApiWithClusters extends TestAsyncAdminBase
       ConnectionFactory.createAsyncConnection(TEST_UTIL2.getConfiguration()).get();
     admin2 = connection.getAdmin();
 
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(TEST_UTIL2.getClusterKey());
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(TEST_UTIL2.getClusterKey()).build();
     ASYNC_CONN.getAdmin().addReplicationPeer(ID_SECOND, rpc).join();
   }
 
@@ -231,30 +232,30 @@ public class TestAsyncReplicationAdminApiWithClusters extends TestAsyncAdminBase
     assertFalse("Table should not exists in the peer cluster",
       admin2.tableExists(tableName2).get());
 
-    Map<TableName, ? extends Collection<String>> tableCfs = new HashMap<>();
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
     tableCfs.put(tableName, null);
-    ReplicationPeerConfig rpc = admin.getReplicationPeerConfig(ID_SECOND).get();
-    rpc.setReplicateAllUserTables(false);
-    rpc.setTableCFsMap(tableCfs);
+    ReplicationPeerConfigBuilder rpcBuilder = ReplicationPeerConfig
+      .newBuilder(admin.getReplicationPeerConfig(ID_SECOND).get())
+      .setReplicateAllUserTables(false)
+      .setTableCFsMap(tableCfs);
     try {
       // Only add tableName to replication peer config
-      admin.updateReplicationPeerConfig(ID_SECOND, rpc).join();
+      admin.updateReplicationPeerConfig(ID_SECOND, rpcBuilder.build()).join();
       admin.enableTableReplication(tableName2).join();
       assertFalse("Table should not be created if user has set table cfs explicitly for the "
           + "peer and this is not part of that collection", admin2.tableExists(tableName2).get());
 
       // Add tableName2 to replication peer config, too
       tableCfs.put(tableName2, null);
-      rpc.setTableCFsMap(tableCfs);
-      admin.updateReplicationPeerConfig(ID_SECOND, rpc).join();
+      rpcBuilder.setTableCFsMap(tableCfs);
+      admin.updateReplicationPeerConfig(ID_SECOND, rpcBuilder.build()).join();
       admin.enableTableReplication(tableName2).join();
       assertTrue(
         "Table should be created if user has explicitly added table into table cfs collection",
         admin2.tableExists(tableName2).get());
     } finally {
-      rpc.setTableCFsMap(null);
-      rpc.setReplicateAllUserTables(true);
-      admin.updateReplicationPeerConfig(ID_SECOND, rpc).join();
+      rpcBuilder.setTableCFsMap(null).setReplicateAllUserTables(true).build();
+      admin.updateReplicationPeerConfig(ID_SECOND, rpcBuilder.build()).join();
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 491612c..99180ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -395,8 +395,8 @@ public class TestReplicaWithCluster {
 
     try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration());
       Admin admin = connection.getAdmin()) {
-      ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-      rpc.setClusterKey(HTU2.getClusterKey());
+      ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+        .setClusterKey(HTU2.getClusterKey()).build();
       admin.addReplicationPeer("2", rpc);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index b2e0e6d..9baa600 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -623,7 +623,8 @@ public class TestMasterReplication {
     try (Connection conn = ConnectionFactory.createConnection(configurations[masterClusterNumber]);
       Admin admin = conn.getAdmin()) {
       admin.addReplicationPeer(id,
-        new ReplicationPeerConfig().setClusterKey(utilities[slaveClusterNumber].getClusterKey()));
+        ReplicationPeerConfig.newBuilder().
+          setClusterKey(utilities[slaveClusterNumber].getClusterKey()).build());
     }
   }
 
@@ -633,9 +634,10 @@ public class TestMasterReplication {
       Admin admin = conn.getAdmin()) {
       admin.addReplicationPeer(
         id,
-        new ReplicationPeerConfig().setClusterKey(utilities[slaveClusterNumber].getClusterKey())
-            .setReplicateAllUserTables(false)
-            .setTableCFsMap(ReplicationPeerConfigUtil.parseTableCFsFromConfig(tableCfs)));
+        ReplicationPeerConfig.newBuilder()
+          .setClusterKey(utilities[slaveClusterNumber].getClusterKey())
+          .setReplicateAllUserTables(false)
+          .setTableCFsMap(ReplicationPeerConfigUtil.parseTableCFsFromConfig(tableCfs)).build());
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index 322db2e..b3e4a1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -145,9 +145,9 @@ public class TestMultiSlaveReplication {
       Table htable2 = utility2.getConnection().getTable(tableName);
       Table htable3 = utility3.getConnection().getTable(tableName);
 
-      ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-      rpc.setClusterKey(utility2.getClusterKey());
-      admin1.addReplicationPeer("1", rpc);
+      ReplicationPeerConfigBuilder rpcBuilder =
+        ReplicationPeerConfig.newBuilder().setClusterKey(utility2.getClusterKey());
+      admin1.addReplicationPeer("1", rpcBuilder.build());
 
       // put "row" and wait 'til it got around, then delete
       putAndWait(row, famName, htable1, htable2);
@@ -163,9 +163,8 @@ public class TestMultiSlaveReplication {
       // after the log was rolled put a new row
       putAndWait(row3, famName, htable1, htable2);
 
-      rpc = new ReplicationPeerConfig();
-      rpc.setClusterKey(utility3.getClusterKey());
-      admin1.addReplicationPeer("2", rpc);
+      rpcBuilder.setClusterKey(utility3.getClusterKey());
+      admin1.addReplicationPeer("2", rpcBuilder.build());
 
       // put a row, check it was replicated to all clusters
       putAndWait(row1, famName, htable1, htable2, htable3);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index 5a6ac0c..b972c5f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -142,8 +142,10 @@ public class TestReplicationEndpoint extends TestReplicationBase {
   public void testCustomReplicationEndpoint() throws Exception {
     // test installing a custom replication endpoint other than the default one.
     hbaseAdmin.addReplicationPeer("testCustomReplicationEndpoint",
-        new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
-            .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()));
+      ReplicationPeerConfig.newBuilder()
+        .setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
+        .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName())
+        .build());
 
     // check whether the class has been constructed and started
     Waiter.waitFor(CONF1, 60000, new Waiter.Predicate<Exception>() {
@@ -184,8 +186,10 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     int peerCount = hbaseAdmin.listReplicationPeers().size();
     final String id = "testReplicationEndpointReturnsFalseOnReplicate";
     hbaseAdmin.addReplicationPeer(id,
-      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
-        .setReplicationEndpointImpl(ReplicationEndpointReturningFalse.class.getName()));
+      ReplicationPeerConfig.newBuilder()
+        .setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
+        .setReplicationEndpointImpl(ReplicationEndpointReturningFalse.class.getName())
+        .build());
     // This test is flakey and then there is so much stuff flying around in here its, hard to
     // debug.  Peer needs to be up for the edit to make it across. This wait on
     // peer count seems to be a hack that has us not progress till peer is up.
@@ -236,8 +240,10 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     }
 
     hbaseAdmin.addReplicationPeer(id,
-        new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF2))
-            .setReplicationEndpointImpl(InterClusterReplicationEndpointForTest.class.getName()));
+      ReplicationPeerConfig.newBuilder()
+        .setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF2))
+        .setReplicationEndpointImpl(InterClusterReplicationEndpointForTest.class.getName())
+        .build());
 
     final int numEdits = totEdits;
     Waiter.waitFor(CONF1, 30000, new Waiter.ExplainingPredicate<Exception>() {
@@ -260,13 +266,15 @@ public class TestReplicationEndpoint extends TestReplicationBase {
 
   @Test
   public void testWALEntryFilterFromReplicationEndpoint() throws Exception {
-    ReplicationPeerConfig rpc =
-      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
-        .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName());
-    // test that we can create mutliple WALFilters reflectively
-    rpc.getConfiguration().put(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
-      EverythingPassesWALEntryFilter.class.getName() + "," +
-        EverythingPassesWALEntryFilterSubclass.class.getName());
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
+      .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName())
+      // test that we can create mutliple WALFilters reflectively
+      .putConfiguration(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
+        EverythingPassesWALEntryFilter.class.getName() + "," +
+          EverythingPassesWALEntryFilterSubclass.class.getName())
+      .build();
+
     hbaseAdmin.addReplicationPeer("testWALEntryFilterFromReplicationEndpoint", rpc);
     // now replicate some data.
     try (Connection connection = ConnectionFactory.createConnection(CONF1)) {
@@ -290,23 +298,25 @@ public class TestReplicationEndpoint extends TestReplicationBase {
 
   @Test(expected = IOException.class)
   public void testWALEntryFilterAddValidation() throws Exception {
-    ReplicationPeerConfig rpc =
-      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
-        .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName());
-    // test that we can create mutliple WALFilters reflectively
-    rpc.getConfiguration().put(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
-      "IAmNotARealWalEntryFilter");
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
+      .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName())
+      // test that we can create mutliple WALFilters reflectively
+      .putConfiguration(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
+        "IAmNotARealWalEntryFilter")
+      .build();
     hbaseAdmin.addReplicationPeer("testWALEntryFilterAddValidation", rpc);
   }
 
   @Test(expected = IOException.class)
   public void testWALEntryFilterUpdateValidation() throws Exception {
-    ReplicationPeerConfig rpc =
-      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
-        .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName());
-    // test that we can create mutliple WALFilters reflectively
-    rpc.getConfiguration().put(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
-      "IAmNotARealWalEntryFilter");
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(ZKConfig.getZooKeeperClusterKey(CONF1))
+      .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName())
+      // test that we can create mutliple WALFilters reflectively
+      .putConfiguration(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY,
+        "IAmNotARealWalEntryFilter")
+      .build();
     hbaseAdmin.updateReplicationPeerConfig("testWALEntryFilterUpdateValidation", rpc);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
index d416e09..d61966f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
@@ -143,8 +143,8 @@ public class TestReplicationWithTags {
 
     connection1 = ConnectionFactory.createConnection(conf1);
     replicationAdmin = connection1.getAdmin();
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(utility2.getClusterKey());
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(utility2.getClusterKey()).build();
     replicationAdmin.addReplicationPeer("2", rpc);
 
     TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java
index 1538fa3..f528bda 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalReplicationThrottler.java
@@ -101,8 +101,8 @@ public class TestGlobalReplicationThrottler {
     utility2.setZkCluster(miniZK);
     new ZKWatcher(conf2, "cluster2", null, true);
 
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(utility2.getClusterKey());
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(utility2.getClusterKey()).build();
 
     utility1.startMiniCluster();
     utility2.startMiniCluster();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java
index bfdbb88..ce47f0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicator.java
@@ -71,8 +71,8 @@ public class TestReplicator extends TestReplicationBase {
 
     // Replace the peer set up for us by the base class with a wrapper for this test
     hbaseAdmin.addReplicationPeer("testReplicatorBatching",
-      new ReplicationPeerConfig().setClusterKey(UTIL2.getClusterKey())
-          .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()));
+      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
+          .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()).build());
 
     ReplicationEndpointForTest.setBatchCount(0);
     ReplicationEndpointForTest.setEntriesCount(0);
@@ -120,8 +120,9 @@ public class TestReplicator extends TestReplicationBase {
 
     // Replace the peer set up for us by the base class with a wrapper for this test
     hbaseAdmin.addReplicationPeer("testReplicatorWithErrors",
-      new ReplicationPeerConfig().setClusterKey(UTIL2.getClusterKey())
-          .setReplicationEndpointImpl(FailureInjectingReplicationEndpointForTest.class.getName()));
+      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
+        .setReplicationEndpointImpl(FailureInjectingReplicationEndpointForTest.class.getName())
+        .build());
 
     FailureInjectingReplicationEndpointForTest.setBatchCount(0);
     FailureInjectingReplicationEndpointForTest.setEntriesCount(0);
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 1727617..905cb48 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
@@ -3056,7 +3056,8 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preUpdateReplicationPeerConfig(
-          ObserverContextImpl.createAndPrepare(CP_ENV), "test", new ReplicationPeerConfig());
+          ObserverContextImpl.createAndPrepare(CP_ENV), "test",
+          ReplicationPeerConfig.newBuilder().build());
         return null;
       }
     };
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
index 10b8cf5..31f219c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
@@ -137,8 +137,9 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
     TEST_UTIL1.startMiniCluster(1);
 
     admin = TEST_UTIL.getAdmin();
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(TEST_UTIL1.getClusterKey());
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(TEST_UTIL1.getClusterKey())
+      .build();
     admin.addReplicationPeer("2", rpc);
 
     TableDescriptor tableDescriptor =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
index 012c9aa..b843f6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
@@ -190,8 +190,8 @@ public class TestVisibilityLabelsReplication {
     TEST_UTIL1.startMiniCluster(1);
 
     admin = TEST_UTIL.getAdmin();
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(TEST_UTIL1.getClusterKey());
+    ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+      .setClusterKey(TEST_UTIL1.getClusterKey()).build();
     admin.addReplicationPeer("2", rpc);
 
     Admin hBaseAdmin = TEST_UTIL.getAdmin();