You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2022/06/15 02:55:29 UTC

[hbase] 02/02: HBASE-26167 Allow users to not start zookeeper and dfs cluster when using TestingHBaseCluster (#4534)

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

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

commit 7fc167493398436dccc4bc7864f39b278f386119
Author: xicm <xi...@gmail.com>
AuthorDate: Wed Jun 15 10:48:48 2022 +0800

    HBASE-26167 Allow users to not start zookeeper and dfs cluster when using TestingHBaseCluster (#4534)
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Yu Li <li...@apache.org>
---
 .../hbase/testing/TestingHBaseClusterImpl.java     |  24 ++++-
 .../hbase/testing/TestingHBaseClusterOption.java   |  31 +++++-
 ...TestTestingHBaseClusterReplicationShareDfs.java |  58 +++++++++++
 .../TestTestingHBaseClusterReplicationShareZk.java |  58 +++++++++++
 ...stTestingHBaseClusterReplicationShareZkDfs.java |  63 +++++++++++
 ...tTestingHBaseClusterReplicationTwoClusters.java |  46 ++++++++
 .../TestingHBaseClusterReplicationTestBase.java    | 116 +++++++++++++++++++++
 7 files changed, 393 insertions(+), 3 deletions(-)

diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterImpl.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterImpl.java
index 8724d9f6c3..35198e770d 100644
--- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterImpl.java
+++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterImpl.java
@@ -19,12 +19,15 @@ package org.apache.hadoop.hbase.testing;
 
 import java.util.List;
 import java.util.Optional;
+import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.StartTestingClusterOption;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -46,6 +49,10 @@ class TestingHBaseClusterImpl implements TestingHBaseCluster {
 
   private final StartTestingClusterOption option;
 
+  private final String externalDfsUri;
+
+  private final String externalZkConnectString;
+
   private final ExecutorService executor = Executors.newCachedThreadPool(new ThreadFactoryBuilder()
     .setNameFormat(getClass().getSuperclass() + "-%d").setDaemon(true).build());
 
@@ -56,6 +63,8 @@ class TestingHBaseClusterImpl implements TestingHBaseCluster {
   TestingHBaseClusterImpl(TestingHBaseClusterOption option) {
     this.util = new HBaseTestingUtil(option.conf());
     this.option = option.convert();
+    this.externalDfsUri = option.getExternalDfsUri();
+    this.externalZkConnectString = option.getExternalZkConnectString();
   }
 
   @Override
@@ -137,7 +146,20 @@ class TestingHBaseClusterImpl implements TestingHBaseCluster {
   @Override
   public void start() throws Exception {
     Preconditions.checkState(!miniClusterRunning, "Cluster has already been started");
-    util.startMiniCluster(option);
+    if (externalZkConnectString == null) {
+      util.startMiniZKCluster();
+    } else {
+      Configuration conf = util.getConfiguration();
+      conf.set(HConstants.ZOOKEEPER_QUORUM, externalZkConnectString);
+      conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/" + UUID.randomUUID().toString());
+    }
+    if (externalDfsUri == null) {
+      util.startMiniDFSCluster(option.getNumDataNodes(), option.getDataNodeHosts());
+    } else {
+      Configuration conf = util.getConfiguration();
+      conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, externalDfsUri);
+    }
+    util.startMiniHBaseCluster(option);
     miniClusterRunning = true;
     miniHBaseClusterRunning = true;
   }
diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterOption.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterOption.java
index 01d67384ed..3f57af46e7 100644
--- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterOption.java
+++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterOption.java
@@ -98,12 +98,17 @@ public final class TestingHBaseClusterOption {
    */
   private final boolean createWALDir;
 
+  private final String externalDfsUri;
+
+  private final String externalZkConnectString;
+
   /**
    * Private constructor. Use {@link Builder#build()}.
    */
   private TestingHBaseClusterOption(Configuration conf, int numMasters, int numAlwaysStandByMasters,
     int numRegionServers, List<Integer> rsPorts, int numDataNodes, String[] dataNodeHosts,
-    int numZkServers, boolean createRootDir, boolean createWALDir) {
+    int numZkServers, boolean createRootDir, boolean createWALDir, String externalDfsUri,
+    String externalZkConnectString) {
     this.conf = conf;
     this.numMasters = numMasters;
     this.numAlwaysStandByMasters = numAlwaysStandByMasters;
@@ -114,6 +119,8 @@ public final class TestingHBaseClusterOption {
     this.numZkServers = numZkServers;
     this.createRootDir = createRootDir;
     this.createWALDir = createWALDir;
+    this.externalDfsUri = externalDfsUri;
+    this.externalZkConnectString = externalZkConnectString;
   }
 
   public Configuration conf() {
@@ -156,6 +163,14 @@ public final class TestingHBaseClusterOption {
     return createWALDir;
   }
 
+  public String getExternalDfsUri() {
+    return externalDfsUri;
+  }
+
+  public String getExternalZkConnectString() {
+    return externalZkConnectString;
+  }
+
   @Override
   public String toString() {
     return "StartMiniClusterOption{" + "numMasters=" + numMasters + ", numRegionServers="
@@ -197,6 +212,8 @@ public final class TestingHBaseClusterOption {
     private int numZkServers = 1;
     private boolean createRootDir = false;
     private boolean createWALDir = false;
+    private String externalDfsUri = null;
+    private String externalZkConnectString = null;
 
     private Builder() {
     }
@@ -207,7 +224,7 @@ public final class TestingHBaseClusterOption {
       }
       return new TestingHBaseClusterOption(conf, numMasters, numAlwaysStandByMasters,
         numRegionServers, rsPorts, numDataNodes, dataNodeHosts, numZkServers, createRootDir,
-        createWALDir);
+        createWALDir, externalDfsUri, externalZkConnectString);
     }
 
     public Builder conf(Configuration conf) {
@@ -259,5 +276,15 @@ public final class TestingHBaseClusterOption {
       this.createWALDir = createWALDir;
       return this;
     }
+
+    public Builder useExternalDfs(String uri) {
+      this.externalDfsUri = uri;
+      return this;
+    }
+
+    public Builder useExternalZooKeeper(String connectString) {
+      this.externalZkConnectString = connectString;
+      return this;
+    }
   }
 }
diff --git a/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareDfs.java b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareDfs.java
new file mode 100644
index 0000000000..0925200599
--- /dev/null
+++ b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareDfs.java
@@ -0,0 +1,58 @@
+/*
+ * 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.testing;
+
+import static org.junit.Assert.assertNotEquals;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, LargeTests.class })
+public class TestTestingHBaseClusterReplicationShareDfs
+  extends TestingHBaseClusterReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTestingHBaseClusterReplicationShareDfs.class);
+
+  private HBaseTestingUtil util = new HBaseTestingUtil();
+
+  @Override
+  protected void startClusters() throws Exception {
+    util.startMiniDFSCluster(1);
+    String dfsUri = util.getDFSCluster().getFileSystem().getUri().toString();
+    sourceCluster = TestingHBaseCluster
+      .create(TestingHBaseClusterOption.builder().useExternalDfs(dfsUri).build());
+    sourceCluster.start();
+    peerCluster = TestingHBaseCluster
+      .create(TestingHBaseClusterOption.builder().useExternalDfs(dfsUri).build());
+    peerCluster.start();
+    assertNotEquals(sourceCluster.getConf().get(HConstants.HBASE_DIR),
+      peerCluster.getConf().get(HConstants.HBASE_DIR));
+  }
+
+  @Override
+  protected void stopClusters() throws Exception {
+    util.shutdownMiniDFSCluster();
+  }
+}
diff --git a/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareZk.java b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareZk.java
new file mode 100644
index 0000000000..b941b9b479
--- /dev/null
+++ b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareZk.java
@@ -0,0 +1,58 @@
+/*
+ * 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.testing;
+
+import static org.junit.Assert.assertNotEquals;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, LargeTests.class })
+public class TestTestingHBaseClusterReplicationShareZk
+  extends TestingHBaseClusterReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTestingHBaseClusterReplicationShareZk.class);
+
+  private HBaseTestingUtil util = new HBaseTestingUtil();
+
+  @Override
+  protected void startClusters() throws Exception {
+    util.startMiniZKCluster();
+    String zkConnectString = util.getZkCluster().getAddress().toString();
+    sourceCluster = TestingHBaseCluster
+      .create(TestingHBaseClusterOption.builder().useExternalZooKeeper(zkConnectString).build());
+    sourceCluster.start();
+    peerCluster = TestingHBaseCluster
+      .create(TestingHBaseClusterOption.builder().useExternalZooKeeper(zkConnectString).build());
+    peerCluster.start();
+    assertNotEquals(sourceCluster.getConf().get(HConstants.ZOOKEEPER_ZNODE_PARENT),
+      peerCluster.getConf().get(HConstants.ZOOKEEPER_ZNODE_PARENT));
+  }
+
+  @Override
+  protected void stopClusters() throws Exception {
+    util.shutdownMiniZKCluster();
+  }
+}
diff --git a/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareZkDfs.java b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareZkDfs.java
new file mode 100644
index 0000000000..ba2c0e75db
--- /dev/null
+++ b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationShareZkDfs.java
@@ -0,0 +1,63 @@
+/*
+ * 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.testing;
+
+import static org.junit.Assert.assertNotEquals;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, LargeTests.class })
+public class TestTestingHBaseClusterReplicationShareZkDfs
+  extends TestingHBaseClusterReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTestingHBaseClusterReplicationShareZkDfs.class);
+
+  private HBaseTestingUtil util = new HBaseTestingUtil();
+
+  @Override
+  protected void startClusters() throws Exception {
+    util.startMiniZKCluster();
+    util.startMiniDFSCluster(1);
+    String zkConnectString = util.getZkCluster().getAddress().toString();
+    String dfsUri = util.getDFSCluster().getFileSystem().getUri().toString();
+    sourceCluster = TestingHBaseCluster.create(TestingHBaseClusterOption.builder()
+      .useExternalZooKeeper(zkConnectString).useExternalDfs(dfsUri).build());
+    sourceCluster.start();
+    peerCluster = TestingHBaseCluster.create(TestingHBaseClusterOption.builder()
+      .useExternalZooKeeper(zkConnectString).useExternalDfs(dfsUri).build());
+    peerCluster.start();
+    assertNotEquals(sourceCluster.getConf().get(HConstants.ZOOKEEPER_ZNODE_PARENT),
+      peerCluster.getConf().get(HConstants.ZOOKEEPER_ZNODE_PARENT));
+    assertNotEquals(sourceCluster.getConf().get(HConstants.HBASE_DIR),
+      peerCluster.getConf().get(HConstants.HBASE_DIR));
+  }
+
+  @Override
+  protected void stopClusters() throws Exception {
+    util.shutdownMiniDFSCluster();
+    util.shutdownMiniZKCluster();
+  }
+}
diff --git a/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationTwoClusters.java b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationTwoClusters.java
new file mode 100644
index 0000000000..4c57edd095
--- /dev/null
+++ b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestTestingHBaseClusterReplicationTwoClusters.java
@@ -0,0 +1,46 @@
+/*
+ * 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.testing;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, LargeTests.class })
+public class TestTestingHBaseClusterReplicationTwoClusters
+  extends TestingHBaseClusterReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTestingHBaseClusterReplicationTwoClusters.class);
+
+  @Override
+  protected void startClusters() throws Exception {
+    sourceCluster = TestingHBaseCluster.create(TestingHBaseClusterOption.builder().build());
+    sourceCluster.start();
+    peerCluster = TestingHBaseCluster.create(TestingHBaseClusterOption.builder().build());
+    peerCluster.start();
+  }
+
+  @Override
+  protected void stopClusters() throws Exception {
+  }
+
+}
diff --git a/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterReplicationTestBase.java b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterReplicationTestBase.java
new file mode 100644
index 0000000000..445c35e3c3
--- /dev/null
+++ b/hbase-testing-util/src/test/java/org/apache/hadoop/hbase/testing/TestingHBaseClusterReplicationTestBase.java
@@ -0,0 +1,116 @@
+/*
+ * 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.testing;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+/**
+ * Setup two clusters for replication.
+ */
+public abstract class TestingHBaseClusterReplicationTestBase {
+
+  protected TestingHBaseCluster sourceCluster;
+
+  protected TestingHBaseCluster peerCluster;
+
+  private Connection sourceConn;
+
+  private Connection peerConn;
+
+  private TableName tableName = TableName.valueOf("test_rep");
+
+  private byte[] family = Bytes.toBytes("family");
+
+  private String peerId = "peer_id";
+
+  private String getPeerClusterKey() {
+    return ZKConfig.getZooKeeperClusterKey(peerCluster.getConf());
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    startClusters();
+    sourceConn = ConnectionFactory.createConnection(sourceCluster.getConf());
+    peerConn = ConnectionFactory.createConnection(peerCluster.getConf());
+    TableDescriptor desc =
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(family).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build();
+    try (Admin admin = sourceConn.getAdmin()) {
+      admin.createTable(desc);
+      admin.addReplicationPeer(peerId, ReplicationPeerConfig.newBuilder()
+        .setClusterKey(getPeerClusterKey()).setReplicateAllUserTables(true).build());
+    }
+    try (Admin admin = peerConn.getAdmin()) {
+      admin.createTable(desc);
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    Closeables.close(sourceConn, true);
+    Closeables.close(peerConn, true);
+    if (sourceCluster != null) {
+      sourceCluster.stop();
+    }
+    if (peerCluster != null) {
+      peerCluster.stop();
+    }
+    stopClusters();
+  }
+
+  @Test
+  public void testReplication() throws IOException {
+    byte[] row = Bytes.toBytes("row");
+    byte[] qual = Bytes.toBytes("qual");
+    byte[] value = Bytes.toBytes("value");
+    try (Table sourceTable = sourceConn.getTable(tableName);
+      Table peerTable = peerConn.getTable(tableName);) {
+      sourceTable.put(new Put(row).addColumn(family, qual, value));
+      Waiter.waitFor(sourceCluster.getConf(), 30000,
+        () -> peerTable.exists(new Get(row).addColumn(family, qual)));
+      byte[] actual = peerTable.get(new Get(row)).getValue(family, qual);
+      assertArrayEquals(value, actual);
+    }
+  }
+
+  protected abstract void startClusters() throws Exception;
+
+  protected abstract void stopClusters() throws Exception;
+}