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/23 04:07:26 UTC

hbase git commit: HBASE-20612 TestReplicationKillSlaveRSWithSeparateOldWALs sometimes fail because it uses an expired cluster conn

Repository: hbase
Updated Branches:
  refs/heads/master dace8ff2a -> 5721150c6


HBASE-20612 TestReplicationKillSlaveRSWithSeparateOldWALs sometimes fail because it uses an expired cluster conn


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

Branch: refs/heads/master
Commit: 5721150c6d910ef95c30c0dd3126f58b6096f1b5
Parents: dace8ff
Author: huzheng <op...@gmail.com>
Authored: Tue May 22 19:39:28 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Wed May 23 12:07:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/client/RpcRetryingCallerImpl.java     |  1 -
 .../replication/TestReplicationKillRS.java      | 98 ++++++++++----------
 2 files changed, 50 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5721150c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
index 2d05dbb..96bb45d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
@@ -108,7 +108,6 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
       } catch (PreemptiveFastFailException e) {
         throw e;
       } catch (Throwable t) {
-        Throwable e = t.getCause();
         ExceptionUtil.rethrowIfInterrupt(t);
         Throwable cause = t.getCause();
         if (cause instanceof DoNotRetryIOException) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5721150c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
index 643b629..5b4fa2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
@@ -22,9 +22,12 @@ import static org.junit.Assert.fail;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.UnknownScannerException;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 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.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.junit.ClassRule;
@@ -32,7 +35,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 TestReplicationKillRS extends TestReplicationBase {
 
   @ClassRule
@@ -42,38 +45,34 @@ public class TestReplicationKillRS extends TestReplicationBase {
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationKillRS.class);
 
   /**
-   * Load up 1 tables over 2 region servers and kill a source during
-   * the upload. The failover happens internally.
-   *
-   * WARNING this test sometimes fails because of HBASE-3515
-   *
-   * @throws Exception
+   * Load up 1 tables over 2 region servers and kill a source during the upload. The failover
+   * happens internally. WARNING this test sometimes fails because of HBASE-3515
    */
   public void loadTableAndKillRS(HBaseTestingUtility util) throws Exception {
     // killing the RS with hbase:meta can result into failed puts until we solve
     // IO fencing
-    int rsToKill1 =
-        util.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
+    int rsToKill1 = util.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
 
     // Takes about 20 secs to run the full loading, kill around the middle
     Thread killer = killARegionServer(util, 5000, rsToKill1);
-
-    LOG.info("Start loading table");
-    int initialCount = utility1.loadTable(htable1, famName);
-    LOG.info("Done loading table");
-    killer.join(5000);
-    LOG.info("Done waiting for threads");
-
     Result[] res;
-    while (true) {
-      try {
-        Scan scan = new Scan();
-        ResultScanner scanner = htable1.getScanner(scan);
-        res = scanner.next(initialCount);
-        scanner.close();
-        break;
-      } catch (UnknownScannerException ex) {
-        LOG.info("Cluster wasn't ready yet, restarting scanner");
+    int initialCount;
+    try (Connection conn = ConnectionFactory.createConnection(conf1)) {
+      try (Table table = conn.getTable(tableName)) {
+        LOG.info("Start loading table");
+        initialCount = utility1.loadTable(table, famName);
+        LOG.info("Done loading table");
+        killer.join(5000);
+        LOG.info("Done waiting for threads");
+
+        while (true) {
+          try (ResultScanner scanner = table.getScanner(new Scan())) {
+            res = scanner.next(initialCount);
+            break;
+          } catch (UnknownScannerException ex) {
+            LOG.info("Cluster wasn't ready yet, restarting scanner");
+          }
+        }
       }
     }
     // Test we actually have all the rows, we may miss some because we
@@ -85,36 +84,39 @@ public class TestReplicationKillRS extends TestReplicationBase {
     }
 
     int lastCount = 0;
-
     final long start = System.currentTimeMillis();
     int i = 0;
-    while (true) {
-      if (i==NB_RETRIES-1) {
-        fail("Waited too much time for queueFailover replication. " +
-            "Waited "+(System.currentTimeMillis() - start)+"ms.");
-      }
-      Scan scan2 = new Scan();
-      ResultScanner scanner2 = htable2.getScanner(scan2);
-      Result[] res2 = scanner2.next(initialCount * 2);
-      scanner2.close();
-      if (res2.length < initialCount) {
-        if (lastCount < res2.length) {
-          i--; // Don't increment timeout if we make progress
-        } else {
-          i++;
+    try (Connection conn = ConnectionFactory.createConnection(conf2)) {
+      try (Table table = conn.getTable(tableName)) {
+        while (true) {
+          if (i == NB_RETRIES - 1) {
+            fail("Waited too much time for queueFailover replication. " + "Waited "
+                + (System.currentTimeMillis() - start) + "ms.");
+          }
+          Result[] res2;
+          try (ResultScanner scanner = table.getScanner(new Scan())) {
+            res2 = scanner.next(initialCount * 2);
+          }
+          if (res2.length < initialCount) {
+            if (lastCount < res2.length) {
+              i--; // Don't increment timeout if we make progress
+            } else {
+              i++;
+            }
+            lastCount = res2.length;
+            LOG.info(
+              "Only got " + lastCount + " rows instead of " + initialCount + " current i=" + i);
+            Thread.sleep(SLEEP_TIME * 2);
+          } else {
+            break;
+          }
         }
-        lastCount = res2.length;
-        LOG.info("Only got " + lastCount + " rows instead of " +
-            initialCount + " current i=" + i);
-        Thread.sleep(SLEEP_TIME*2);
-      } else {
-        break;
       }
     }
   }
 
-  private static Thread killARegionServer(final HBaseTestingUtility utility,
-                                          final long timeout, final int rs) {
+  private static Thread killARegionServer(final HBaseTestingUtility utility, final long timeout,
+      final int rs) {
     Thread killer = new Thread() {
       @Override
       public void run() {