You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by op...@apache.org on 2018/05/14 11:38:44 UTC

[1/2] hbase git commit: HBASE-20128 Add new UTs which extends the old replication UTs but set replication scope to SERIAL

Repository: hbase
Updated Branches:
  refs/heads/branch-2 aa8ceb816 -> 4b0ac73f5


HBASE-20128 Add new UTs which extends the old replication UTs but set replication scope to SERIAL


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

Branch: refs/heads/branch-2
Commit: be3df29cef84d0e34f932b36a649912a7fcc0f23
Parents: aa8ceb8
Author: huzheng <op...@gmail.com>
Authored: Thu Apr 12 15:17:32 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Mon May 14 19:32:39 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationPeers.java     |  7 ---
 .../TestReplicationAdminWithClusters.java       | 10 ++++-
 .../replication/TestMasterReplication.java      |  2 +-
 .../replication/TestMultiSlaveReplication.java  |  2 +-
 .../replication/TestNamespaceReplication.java   | 26 +++++++++--
 .../hbase/replication/TestReplicationBase.java  | 46 ++++++++++---------
 ...estReplicationChangingPeerRegionservers.java | 30 ++++++++++---
 .../TestReplicationDroppedTables.java           |  6 +++
 .../replication/TestReplicationEndpoint.java    |  8 +++-
 .../TestReplicationKillMasterRS.java            |  6 +--
 .../TestReplicationKillMasterRSCompressed.java  |  2 +-
 ...licationKillMasterRSWithSeparateOldWALs.java | 47 ++++++++++++++++++++
 .../replication/TestReplicationKillSlaveRS.java |  5 +--
 ...plicationKillSlaveRSWithSeparateOldWALs.java | 47 ++++++++++++++++++++
 .../replication/TestReplicationSmallTests.java  | 24 +++++++++-
 .../replication/TestReplicationSyncUpTool.java  |  9 +++-
 16 files changed, 222 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index 24042c6..e754e38 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -127,16 +127,9 @@ public class ReplicationPeers {
   }
 
   /**
-<<<<<<< 2bb2fd611d4b88c724a2b561f10433b56c6fd3dd
-   * Update the peerConfig for the a given peer cluster
-   * @param id a short that identifies the cluster
-   * @param peerConfig new config for the peer cluster
-   * @throws ReplicationException
-=======
    * Helper method to connect to a peer
    * @param peerId peer's identifier
    * @return object representing the peer
->>>>>>> HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface
    */
   private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index ff422aa..268fe00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -96,12 +96,13 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     admin1.modifyTable(tableName, table);
     admin1.enableTable(tableName);
 
-
     admin1.disableTableReplication(tableName);
     table = admin1.getTableDescriptor(tableName);
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
     }
+
+    admin1.deleteColumnFamily(table.getTableName(), f.getName());
   }
 
   @Test
@@ -158,6 +159,9 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
+
+    admin1.deleteColumnFamily(tableName, f.getName());
+    admin2.deleteColumnFamily(tableName, f.getName());
   }
 
   @Test
@@ -252,12 +256,14 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     rpc.getConfiguration().put("key1", "value2");
     admin.updatePeerConfig(peerId, rpc);
     if (!TestUpdatableReplicationEndpoint.hasCalledBack()) {
-      synchronized(TestUpdatableReplicationEndpoint.class) {
+      synchronized (TestUpdatableReplicationEndpoint.class) {
         TestUpdatableReplicationEndpoint.class.wait(2000L);
       }
     }
 
     assertEquals(true, TestUpdatableReplicationEndpoint.hasCalledBack());
+
+    admin.removePeer(peerId);
   }
 
   public static class TestUpdatableReplicationEndpoint extends BaseReplicationEndpoint {

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
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 de0b94f..37ca7dc 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
@@ -90,7 +90,7 @@ public class TestMasterReplication {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMasterReplication.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationBase.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterReplication.class);
 
   private Configuration baseConfiguration;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
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 f30c48d..225ca7f 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
@@ -58,7 +58,7 @@ public class TestMultiSlaveReplication {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMultiSlaveReplication.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationBase.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestMultiSlaveReplication.class);
 
   private static Configuration conf1;
   private static Configuration conf2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
index 37da482..d8a02c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
@@ -49,10 +49,17 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category({MediumTests.class})
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+
+@RunWith(Parameterized.class)
+@Category({ MediumTests.class })
 public class TestNamespaceReplication extends TestReplicationBase {
 
   @ClassRule
@@ -77,6 +84,19 @@ public class TestNamespaceReplication extends TestReplicationBase {
   private static Admin admin1;
   private static Admin admin2;
 
+  @Parameter
+  public boolean serialPeer;
+
+  @Override
+  protected boolean isSerialPeer() {
+    return serialPeer;
+  }
+
+  @Parameters(name = "{index}: serialPeer={0}")
+  public static List<Boolean> parameters() {
+    return ImmutableList.of(true, false);
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TestReplicationBase.setUpBeforeClass();
@@ -224,7 +244,7 @@ public class TestNamespaceReplication extends TestReplicationBase {
           assertArrayEquals(val, res.value());
           break;
         }
-        Thread.sleep(SLEEP_TIME);
+        Thread.sleep(10 * SLEEP_TIME);
       }
     }
   }
@@ -244,7 +264,7 @@ public class TestNamespaceReplication extends TestReplicationBase {
         } else {
           break;
         }
-        Thread.sleep(SLEEP_TIME);
+        Thread.sleep(10 * SLEEP_TIME);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index a0200d0..b2c5aef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
@@ -50,13 +49,12 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.runners.Parameterized.Parameter;
-import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,11 +65,6 @@ import org.slf4j.LoggerFactory;
  * All other tests should have their own classes and extend this one
  */
 public class TestReplicationBase {
-/*
-  {
-    ((Log4JLogger) ReplicationSource.LOG).getLogger().setLevel(Level.ALL);
-  }*/
-
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationBase.class);
 
   protected static Configuration conf1 = HBaseConfiguration.create();
@@ -100,13 +93,10 @@ public class TestReplicationBase {
   protected static final byte[] famName = Bytes.toBytes("f");
   protected static final byte[] row = Bytes.toBytes("row");
   protected static final byte[] noRepfamName = Bytes.toBytes("norep");
+  protected static final String PEER_ID2 = "2";
 
-  @Parameter
-  public static boolean seperateOldWALs;
-
-  @Parameters
-  public static List<Boolean> params() {
-    return Arrays.asList(false, true);
+  protected boolean isSerialPeer() {
+    return false;
   }
 
   protected final void cleanUp() throws IOException, InterruptedException {
@@ -197,9 +187,6 @@ public class TestReplicationBase {
     conf1.setBoolean("replication.source.eof.autorecovery", true);
     conf1.setLong("hbase.serial.replication.waiting.ms", 100);
 
-    // Parameter config
-    conf1.setBoolean(AbstractFSWALProvider.SEPARATE_OLDLOGDIR, seperateOldWALs);
-
     utility1 = new HBaseTestingUtility(conf1);
     utility1.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
@@ -227,10 +214,7 @@ public class TestReplicationBase {
     // as a component in deciding maximum number of parallel batches to send to the peer cluster.
     utility2.startMiniCluster(4);
 
-    ReplicationPeerConfig rpc =
-        ReplicationPeerConfig.newBuilder().setClusterKey(utility2.getClusterKey()).build();
     hbaseAdmin = ConnectionFactory.createConnection(conf1).getAdmin();
-    hbaseAdmin.addReplicationPeer("2", rpc);
 
     TableDescriptor table = TableDescriptorBuilder.newBuilder(tableName)
         .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName).setMaxVersions(100)
@@ -254,6 +238,26 @@ public class TestReplicationBase {
     htable2 = connection2.getTable(tableName);
   }
 
+  private boolean peerExist(String peerId) throws IOException {
+    return hbaseAdmin.listReplicationPeers().stream().anyMatch(p -> peerId.equals(p.getPeerId()));
+  }
+
+  @Before
+  public void setUpBase() throws IOException {
+    if (!peerExist(PEER_ID2)) {
+      ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
+          .setClusterKey(utility2.getClusterKey()).setSerial(isSerialPeer()).build();
+      hbaseAdmin.addReplicationPeer(PEER_ID2, rpc);
+    }
+  }
+
+  @After
+  public void tearDownBase() throws IOException {
+    if (peerExist(PEER_ID2)) {
+      hbaseAdmin.removeReplicationPeer(PEER_ID2);
+    }
+  }
+
   protected static void runSimplePutDeleteTest() throws IOException, InterruptedException {
     Put put = new Put(row);
     put.addColumn(famName, row, row);

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
index 5492cf6..b94b443 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.List;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.client.Get;
@@ -37,13 +39,20 @@ import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+
 /**
  * Test handling of changes to the number of a peer's regionservers.
  */
-@Category({ReplicationTests.class, LargeTests.class})
+@RunWith(Parameterized.class)
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationChangingPeerRegionservers extends TestReplicationBase {
 
   @ClassRule
@@ -53,6 +62,19 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
   private static final Logger LOG =
       LoggerFactory.getLogger(TestReplicationChangingPeerRegionservers.class);
 
+  @Parameter
+  public boolean serialPeer;
+
+  @Override
+  protected boolean isSerialPeer() {
+    return serialPeer;
+  }
+
+  @Parameters(name = "{index}: serialPeer={0}")
+  public static List<Boolean> parameters() {
+    return ImmutableList.of(true, false);
+  }
+
   /**
    * @throws java.lang.Exception
    */
@@ -60,8 +82,8 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
   public void setUp() throws Exception {
     // Starting and stopping replication can make us miss new logs,
     // rolling like this makes sure the most recent one gets added to the queue
-    for (JVMClusterUtil.RegionServerThread r :
-                          utility1.getHBaseCluster().getRegionServerThreads()) {
+    for (JVMClusterUtil.RegionServerThread r : utility1.getHBaseCluster()
+        .getRegionServerThreads()) {
       utility1.getAdmin().rollWALWriter(r.getRegionServer().getServerName());
     }
     utility1.deleteTableData(tableName);
@@ -94,7 +116,6 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
 
   @Test
   public void testChangingNumberOfPeerRegionServers() throws IOException, InterruptedException {
-
     LOG.info("testSimplePutDelete");
     MiniHBaseCluster peerCluster = utility2.getMiniHBaseCluster();
     int numRS = peerCluster.getRegionServerThreads().size();
@@ -116,7 +137,6 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
     assertEquals(numRS, peerCluster.getRegionServerThreads().size());
 
     doPutTest(Bytes.toBytes(3));
-
   }
 
   private void doPutTest(byte[] row) throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
index ae548db..4022195 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
@@ -126,6 +126,12 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
       admin2.createNamespace(NamespaceDescriptor.create("NS").build());
     }
     testEditsBehindDroppedTable(true, "NS:test_dropped");
+    try (Admin admin1 = connection1.getAdmin()) {
+      admin1.deleteNamespace("NS");
+    }
+    try (Admin admin2 = connection2.getAdmin()) {
+      admin2.deleteNamespace("NS");
+    }
   }
 
   private void testEditsBehindDroppedTable(boolean allowProceeding, String tName) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
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 81d74cc..a3c20d6 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
@@ -65,7 +65,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Tests ReplicationSource and ReplicationEndpoint interactions
  */
-@Category({ReplicationTests.class, MediumTests.class})
+@Category({ ReplicationTests.class, MediumTests.class })
 public class TestReplicationEndpoint extends TestReplicationBase {
 
   @ClassRule
@@ -79,7 +79,6 @@ public class TestReplicationEndpoint extends TestReplicationBase {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TestReplicationBase.setUpBeforeClass();
-    admin.removePeer("2");
     numRegionServers = utility1.getHBaseCluster().getRegionServerThreads().size();
   }
 
@@ -390,6 +389,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     static volatile List<Entry> lastEntries = null;
 
     public ReplicationEndpointForTest() {
+      replicateCount.set(0);
       contructedCount.incrementAndGet();
     }
 
@@ -434,6 +434,10 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     static AtomicInteger replicateCount = new AtomicInteger();
     static boolean failedOnce;
 
+    public InterClusterReplicationEndpointForTest() {
+      replicateCount.set(0);
+    }
+
     @Override
     public boolean replicate(ReplicateContext replicateContext) {
       boolean success = super.replicate(replicateContext);

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java
index 4b8e634..41cc9bc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRS.java
@@ -23,15 +23,12 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 /**
  * Runs the TestReplicationKillRS test and selects the RS to kill in the master cluster
  * Do not add other tests in this class.
  */
-@RunWith(Parameterized.class)
-@Category({ReplicationTests.class, LargeTests.class})
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationKillMasterRS extends TestReplicationKillRS {
 
   @ClassRule
@@ -42,5 +39,4 @@ public class TestReplicationKillMasterRS extends TestReplicationKillRS {
   public void killOneMasterRS() throws Exception {
     loadTableAndKillRS(utility1);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
index 90b9bd8..6cbae83 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
@@ -29,7 +29,7 @@ import org.junit.experimental.categories.Category;
  * Run the same test as TestReplicationKillMasterRS but with WAL compression enabled
  * Do not add other tests in this class.
  */
-@Category({ReplicationTests.class, LargeTests.class})
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationKillMasterRSCompressed extends TestReplicationKillMasterRS {
 
   @ClassRule

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSWithSeparateOldWALs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSWithSeparateOldWALs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSWithSeparateOldWALs.java
new file mode 100644
index 0000000..108f274
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSWithSeparateOldWALs.java
@@ -0,0 +1,47 @@
+/**
+ * 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.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestReplicationKillMasterRSWithSeparateOldWALs extends TestReplicationKillRS {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestReplicationKillMasterRSWithSeparateOldWALs.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.setBoolean(AbstractFSWALProvider.SEPARATE_OLDLOGDIR, true);
+    TestReplicationBase.setUpBeforeClass();
+  }
+
+  @Test
+  public void killOneMasterRS() throws Exception {
+    loadTableAndKillRS(utility1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java
index a6a5738..96630b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRS.java
@@ -23,15 +23,12 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 /**
  * Runs the TestReplicationKillRS test and selects the RS to kill in the slave cluster
  * Do not add other tests in this class.
  */
-@RunWith(Parameterized.class)
-@Category({ReplicationTests.class, LargeTests.class})
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationKillSlaveRS extends TestReplicationKillRS {
 
   @ClassRule

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRSWithSeparateOldWALs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRSWithSeparateOldWALs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRSWithSeparateOldWALs.java
new file mode 100644
index 0000000..a852b81
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillSlaveRSWithSeparateOldWALs.java
@@ -0,0 +1,47 @@
+/**
+ * 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.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestReplicationKillSlaveRSWithSeparateOldWALs extends TestReplicationKillRS {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestReplicationKillSlaveRSWithSeparateOldWALs.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.setBoolean(AbstractFSWALProvider.SEPARATE_OLDLOGDIR, true);
+    TestReplicationBase.setUpBeforeClass();
+  }
+
+  @Test
+  public void killOneSlaveRS() throws Exception {
+    loadTableAndKillRS(utility2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 8d24f5e..2c8dc4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -55,9 +55,16 @@ import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+
+@RunWith(Parameterized.class)
 @Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationSmallTests extends TestReplicationBase {
 
@@ -68,6 +75,19 @@ public class TestReplicationSmallTests extends TestReplicationBase {
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSmallTests.class);
   private static final String PEER_ID = "2";
 
+  @Parameter
+  public boolean serialPeer;
+
+  @Override
+  protected boolean isSerialPeer() {
+    return serialPeer;
+  }
+
+  @Parameters(name = "{index}: serialPeer={0}")
+  public static List<Boolean> parameters() {
+    return ImmutableList.of(true, false);
+  }
+
   @Before
   public void setUp() throws Exception {
     cleanUp();
@@ -316,8 +336,8 @@ public class TestReplicationSmallTests extends TestReplicationBase {
             lastRow = currentRow;
           }
           LOG.error("Last row: " + lastRow);
-          fail("Waited too much time for normal batch replication, " + res.length + " instead of " +
-            NB_ROWS_IN_BIG_BATCH + "; waited=" + (System.currentTimeMillis() - start) + "ms");
+          fail("Waited too much time for normal batch replication, " + res.length + " instead of "
+              + NB_ROWS_IN_BIG_BATCH + "; waited=" + (System.currentTimeMillis() - start) + "ms");
         } else {
           LOG.info("Only got " + res.length + " rows... retrying");
           Thread.sleep(SLEEP_TIME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/be3df29c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index 6c487ad..3d2cdfb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -46,7 +48,7 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category({ReplicationTests.class, LargeTests.class})
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationSyncUpTool extends TestReplicationBase {
 
   @ClassRule
@@ -97,7 +99,12 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
     t2_syncupTarget.addFamily(fam);
     fam = new HColumnDescriptor(noRepfamName);
     t2_syncupTarget.addFamily(fam);
+  }
 
+  @After
+  public void tearDownBase() throws IOException {
+    // Do nothing, just replace the super tearDown. because the super tearDown will use the
+    // out-of-data HBase admin to remove replication peer, which will be result in failure.
   }
 
   /**


[2/2] hbase git commit: HBASE-20560 Revisit the TestReplicationDroppedTables ut

Posted by op...@apache.org.
HBASE-20560 Revisit the TestReplicationDroppedTables ut


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

Branch: refs/heads/branch-2
Commit: 4b0ac73f513eabd116edeffadbfc5885bdfc3780
Parents: be3df29
Author: huzheng <op...@gmail.com>
Authored: Tue May 8 17:39:20 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Mon May 14 19:33:51 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/TestReplicationBase.java  |   4 +-
 .../TestReplicationDroppedTables.java           | 153 +++++++++++--------
 .../replication/TestReplicationSyncUpTool.java  |   3 +-
 3 files changed, 92 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4b0ac73f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index b2c5aef..f96dbe5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -243,7 +243,7 @@ public class TestReplicationBase {
   }
 
   @Before
-  public void setUpBase() throws IOException {
+  public void setUpBase() throws Exception {
     if (!peerExist(PEER_ID2)) {
       ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
           .setClusterKey(utility2.getClusterKey()).setSerial(isSerialPeer()).build();
@@ -252,7 +252,7 @@ public class TestReplicationBase {
   }
 
   @After
-  public void tearDownBase() throws IOException {
+  public void tearDownBase() throws Exception {
     if (peerExist(PEER_ID2)) {
       hbaseAdmin.removeReplicationPeer(PEER_ID2);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b0ac73f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
index 4022195..f280c7c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
@@ -17,29 +17,31 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 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.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 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.ipc.RpcServer;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -55,15 +57,18 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
       HBaseClassTestRule.forClass(TestReplicationDroppedTables.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationDroppedTables.class);
+  private static final int ROWS_COUNT = 1000;
 
   @Before
-  public void setUp() throws Exception {
+  public void setUpBase() throws Exception {
     // Starting and stopping replication can make us miss new logs,
     // rolling like this makes sure the most recent one gets added to the queue
     for (JVMClusterUtil.RegionServerThread r : utility1.getHBaseCluster()
         .getRegionServerThreads()) {
       utility1.getAdmin().rollWALWriter(r.getRegionServer().getServerName());
     }
+    // Initialize the peer after wal rolling, so that we will abandon the stuck WALs.
+    super.setUpBase();
     int rowCount = utility1.countRows(tableName);
     utility1.deleteTableData(tableName);
     // truncating the table will send one Delete per row to the slave cluster
@@ -101,9 +106,8 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
 
   @Test
   public void testEditsStuckBehindDroppedTable() throws Exception {
-    // Sanity check
-    // Make sure by default edits for dropped tables stall the replication queue, even when the
-    // table(s) in question have been deleted on both ends.
+    // Sanity check Make sure by default edits for dropped tables stall the replication queue, even
+    // when the table(s) in question have been deleted on both ends.
     testEditsBehindDroppedTable(false, "test_dropped");
   }
 
@@ -134,6 +138,10 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
     }
   }
 
+  private byte[] generateRowKey(int id) {
+    return Bytes.toBytes(String.format("NormalPut%03d", id));
+  }
+
   private void testEditsBehindDroppedTable(boolean allowProceeding, String tName) throws Exception {
     conf1.setBoolean(HConstants.REPLICATION_DROP_ON_DELETED_TABLE_KEY, allowProceeding);
     conf1.setInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY, 1);
@@ -144,13 +152,14 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
     utility1.startMiniHBaseCluster(1, 1);
 
     TableName tablename = TableName.valueOf(tName);
-    byte[] familyname = Bytes.toBytes("fam");
+    byte[] familyName = Bytes.toBytes("fam");
     byte[] row = Bytes.toBytes("row");
 
-    HTableDescriptor table = new HTableDescriptor(tablename);
-    HColumnDescriptor fam = new HColumnDescriptor(familyname);
-    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    table.addFamily(fam);
+    TableDescriptor table =
+        TableDescriptorBuilder
+            .newBuilder(tablename).setColumnFamily(ColumnFamilyDescriptorBuilder
+                .newBuilder(familyName).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+            .build();
 
     Connection connection1 = ConnectionFactory.createConnection(conf1);
     Connection connection2 = ConnectionFactory.createConnection(conf2);
@@ -163,23 +172,25 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
     utility1.waitUntilAllRegionsAssigned(tablename);
     utility2.waitUntilAllRegionsAssigned(tablename);
 
-    Table lHtable1 = utility1.getConnection().getTable(tablename);
-
     // now suspend replication
-    admin.disablePeer("2");
+    try (Admin admin1 = connection1.getAdmin()) {
+      admin1.disableReplicationPeer(PEER_ID2);
+    }
 
     // put some data (lead with 0 so the edit gets sorted before the other table's edits
-    //   in the replication batch)
-    // write a bunch of edits, making sure we fill a batch
-    byte[] rowKey = Bytes.toBytes(0 + " put on table to be dropped");
-    Put put = new Put(rowKey);
-    put.addColumn(familyname, row, row);
-    lHtable1.put(put);
-
-    for (int i = 0; i < 1000; i++) {
-      rowKey = Bytes.toBytes("NormalPut" + i);
-      put = new Put(rowKey).addColumn(famName, row, row);
-      htable1.put(put);
+    // in the replication batch) write a bunch of edits, making sure we fill a batch
+    try (Table droppedTable = connection1.getTable(tablename)) {
+      byte[] rowKey = Bytes.toBytes(0 + " put on table to be dropped");
+      Put put = new Put(rowKey);
+      put.addColumn(familyName, row, row);
+      droppedTable.put(put);
+    }
+
+    try (Table table1 = connection1.getTable(tableName)) {
+      for (int i = 0; i < ROWS_COUNT; i++) {
+        Put put = new Put(generateRowKey(i)).addColumn(famName, row, row);
+        table1.put(put);
+      }
     }
 
     try (Admin admin1 = connection1.getAdmin()) {
@@ -191,12 +202,15 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
       admin2.deleteTable(tablename);
     }
 
-    admin.enablePeer("2");
+    try (Admin admin1 = connection1.getAdmin()) {
+      admin1.enableReplicationPeer(PEER_ID2);
+    }
+
     if (allowProceeding) {
       // in this we'd expect the key to make it over
-      verifyReplicationProceeded(rowKey);
+      verifyReplicationProceeded();
     } else {
-      verifyReplicationStuck(rowKey);
+      verifyReplicationStuck();
     }
     // just to be safe
     conf1.setBoolean(HConstants.REPLICATION_DROP_ON_DELETED_TABLE_KEY, false);
@@ -213,13 +227,14 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
     utility1.startMiniHBaseCluster(1, 1);
 
     TableName tablename = TableName.valueOf("testdroppedtimed");
-    byte[] familyname = Bytes.toBytes("fam");
+    byte[] familyName = Bytes.toBytes("fam");
     byte[] row = Bytes.toBytes("row");
 
-    HTableDescriptor table = new HTableDescriptor(tablename);
-    HColumnDescriptor fam = new HColumnDescriptor(familyname);
-    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    table.addFamily(fam);
+    TableDescriptor table =
+        TableDescriptorBuilder
+            .newBuilder(tablename).setColumnFamily(ColumnFamilyDescriptorBuilder
+                .newBuilder(familyName).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+            .build();
 
     Connection connection1 = ConnectionFactory.createConnection(conf1);
     Connection connection2 = ConnectionFactory.createConnection(conf2);
@@ -232,23 +247,25 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
     utility1.waitUntilAllRegionsAssigned(tablename);
     utility2.waitUntilAllRegionsAssigned(tablename);
 
-    Table lHtable1 = utility1.getConnection().getTable(tablename);
-
     // now suspend replication
-    admin.disablePeer("2");
+    try (Admin admin1 = connection1.getAdmin()) {
+      admin1.disableReplicationPeer(PEER_ID2);
+    }
 
     // put some data (lead with 0 so the edit gets sorted before the other table's edits
-    // in the replication batch)
-    // write a bunch of edits, making sure we fill a batch
-    byte[] rowKey = Bytes.toBytes(0 + " put on table to be dropped");
-    Put put = new Put(rowKey);
-    put.addColumn(familyname, row, row);
-    lHtable1.put(put);
-
-    for (int i = 0; i < 1000; i++) {
-      rowKey = Bytes.toBytes("NormalPut" + i);
-      put = new Put(rowKey).addColumn(famName, row, row);
-      htable1.put(put);
+    // in the replication batch) write a bunch of edits, making sure we fill a batch
+    try (Table droppedTable = connection1.getTable(tablename)) {
+      byte[] rowKey = Bytes.toBytes(0 + " put on table to be dropped");
+      Put put = new Put(rowKey);
+      put.addColumn(familyName, row, row);
+      droppedTable.put(put);
+    }
+
+    try (Table table1 = connection1.getTable(tableName)) {
+      for (int i = 0; i < ROWS_COUNT; i++) {
+        Put put = new Put(generateRowKey(i)).addColumn(famName, row, row);
+        table1.put(put);
+      }
     }
 
     try (Admin admin2 = connection2.getAdmin()) {
@@ -256,48 +273,56 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
       admin2.deleteTable(tablename);
     }
 
-    admin.enablePeer("2");
     // edit should still be stuck
-
     try (Admin admin1 = connection1.getAdmin()) {
+      // enable the replication peer.
+      admin1.enableReplicationPeer(PEER_ID2);
       // the source table still exists, replication should be stalled
-      verifyReplicationStuck(rowKey);
+      verifyReplicationStuck();
 
       admin1.disableTable(tablename);
       // still stuck, source table still exists
-      verifyReplicationStuck(rowKey);
+      verifyReplicationStuck();
 
       admin1.deleteTable(tablename);
       // now the source table is gone, replication should proceed, the
       // offending edits be dropped
-      verifyReplicationProceeded(rowKey);
+      verifyReplicationProceeded();
     }
     // just to be safe
     conf1.setBoolean(HConstants.REPLICATION_DROP_ON_DELETED_TABLE_KEY, false);
   }
 
-  private void verifyReplicationProceeded(byte[] rowkey) throws Exception {
-    Get get = new Get(rowkey);
+  private boolean peerHasAllNormalRows() throws IOException {
+    try (ResultScanner scanner = htable2.getScanner(new Scan())) {
+      Result[] results = scanner.next(ROWS_COUNT);
+      if (results.length != ROWS_COUNT) {
+        return false;
+      }
+      for (int i = 0; i < results.length; i++) {
+        Assert.assertArrayEquals(generateRowKey(i), results[i].getRow());
+      }
+      return true;
+    }
+  }
+
+  private void verifyReplicationProceeded() throws Exception {
     for (int i = 0; i < NB_RETRIES; i++) {
-      if (i==NB_RETRIES-1) {
+      if (i == NB_RETRIES - 1) {
         fail("Waited too much time for put replication");
       }
-      Result res = htable2.get(get);
-      if (res.size() == 0) {
+      if (!peerHasAllNormalRows()) {
         LOG.info("Row not available");
         Thread.sleep(SLEEP_TIME);
       } else {
-        assertArrayEquals(res.getRow(), rowkey);
         break;
       }
     }
   }
 
-  private void verifyReplicationStuck(byte[] rowkey) throws Exception {
-    Get get = new Get(rowkey);
+  private void verifyReplicationStuck() throws Exception {
     for (int i = 0; i < NB_RETRIES; i++) {
-      Result res = htable2.get(get);
-      if (res.size() >= 1) {
+      if (peerHasAllNormalRows()) {
         fail("Edit should have been stuck behind dropped tables");
       } else {
         LOG.info("Row not replicated, let's wait a bit more...");

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b0ac73f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index 3d2cdfb..745c439 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -102,7 +101,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
   }
 
   @After
-  public void tearDownBase() throws IOException {
+  public void tearDownBase() throws Exception {
     // Do nothing, just replace the super tearDown. because the super tearDown will use the
     // out-of-data HBase admin to remove replication peer, which will be result in failure.
   }