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 2019/05/02 09:05:32 UTC

[hbase] 20/25: HBASE-22297 Fix TestRegionMergeTransitionOnCluster and TestSplitTransactionOnCluster

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

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

commit e950f99ccd4b0550a207e14512d35a6fe74ca70c
Author: zhangduo <zh...@apache.org>
AuthorDate: Tue Apr 23 22:22:39 2019 +0800

    HBASE-22297 Fix TestRegionMergeTransitionOnCluster and TestSplitTransactionOnCluster
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../TestRegionMergeTransactionOnCluster.java          | 19 +++++++++----------
 .../regionserver/TestSplitTransactionOnCluster.java   | 16 +++++++---------
 2 files changed, 16 insertions(+), 19 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index ea93468..7b54ffb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -63,6 +62,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
@@ -312,7 +312,6 @@ public class TestRegionMergeTransactionOnCluster {
     LOG.info("Starting " + name.getMethodName());
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final Admin admin = TEST_UTIL.getAdmin();
-    final int syncWaitTimeout = 10 * 60000; // 10min
 
     try {
       // Create table and load data.
@@ -326,8 +325,8 @@ public class TestRegionMergeTransactionOnCluster {
       am.offlineRegion(b);
       try {
         // Merge offline region. Region a is offline here
-        admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false)
-                .get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+        FutureUtils.get(
+          admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false));
         fail("Offline regions should not be able to merge");
       } catch (DoNotRetryRegionException ie) {
         System.out.println(ie);
@@ -336,21 +335,21 @@ public class TestRegionMergeTransactionOnCluster {
 
       try {
         // Merge the same region: b and b.
-        admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true);
+        FutureUtils
+          .get(admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true));
         fail("A region should not be able to merge with itself, even forcifully");
       } catch (IOException ie) {
         assertTrue("Exception should mention regions not online",
-          StringUtils.stringifyException(ie).contains("region to itself")
-            && ie instanceof MergeRegionException);
+          StringUtils.stringifyException(ie).contains("region to itself") &&
+            ie instanceof MergeRegionException);
       }
 
       try {
         // Merge unknown regions
-        admin.mergeRegionsAsync(Bytes.toBytes("-f1"), Bytes.toBytes("-f2"), true);
+        FutureUtils.get(admin.mergeRegionsAsync(Bytes.toBytes("-f1"), Bytes.toBytes("-f2"), true));
         fail("Unknown region could not be merged");
       } catch (IOException ie) {
-        assertTrue("UnknownRegionException should be thrown",
-          ie instanceof UnknownRegionException);
+        assertTrue("UnknownRegionException should be thrown", ie instanceof UnknownRegionException);
       }
       table.close();
     } finally {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 405819e..d7b2f9e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
@@ -89,6 +90,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.RetryCounter;
@@ -328,7 +330,7 @@ public class TestSplitTransactionOnCluster {
       // We don't roll back here anymore. Instead we fail-fast on construction of the
       // split transaction. Catch the exception instead.
       try {
-        this.admin.splitRegionAsync(hri.getRegionName());
+        FutureUtils.get(this.admin.splitRegionAsync(hri.getRegionName()));
         fail();
       } catch (DoNotRetryRegionException e) {
         // Expected
@@ -510,17 +512,13 @@ public class TestSplitTransactionOnCluster {
   }
 
   /**
-   * Verifies HBASE-5806.  Here the case is that splitting is completed but before the
-   * CJ could remove the parent region the master is killed and restarted.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws NodeExistsException
-   * @throws KeeperException
+   * Verifies HBASE-5806. Here the case is that splitting is completed but before the CJ could
+   * remove the parent region the master is killed and restarted.
    */
   @Test
   public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
       throws IOException, InterruptedException, NodeExistsException,
-      KeeperException, ServiceException {
+      KeeperException, ServiceException, ExecutionException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
     // Create table then get the single region for our new table.
@@ -541,7 +539,7 @@ public class TestSplitTransactionOnCluster {
       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
       printOutRegions(server, "Initial regions: ");
       // Call split.
-      this.admin.splitRegionAsync(hri.getRegionName());
+      this.admin.splitRegionAsync(hri.getRegionName()).get();
       List<HRegion> daughters = checkAndGetDaughters(tableName);
 
       // Before cleanup, get a new master.