You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2020/08/06 07:38:02 UTC

[hbase] branch branch-2.2 updated: HBASE-24819 Fix flaky test TestRaceBetweenSCPAndDTP and TestRaceBetweenSCPAndTRSP for branch-2.2 (#2199)

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

zghao pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 89dc088  HBASE-24819 Fix flaky test TestRaceBetweenSCPAndDTP and TestRaceBetweenSCPAndTRSP for branch-2.2 (#2199)
89dc088 is described below

commit 89dc088a060096c146b417a7a5c5ae162836b613
Author: Guanghao Zhang <zg...@apache.org>
AuthorDate: Thu Aug 6 15:37:39 2020 +0800

    HBASE-24819 Fix flaky test TestRaceBetweenSCPAndDTP and TestRaceBetweenSCPAndTRSP for branch-2.2 (#2199)
    
    Signed-off-by: meiyi <my...@gmail.com>
---
 .../assignment/TestRaceBetweenSCPAndDTP.java       | 41 ++++++++++++++++------
 .../assignment/TestRaceBetweenSCPAndTRSP.java      |  8 +++--
 2 files changed, 36 insertions(+), 13 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java
index 9bfc252..52d6063 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
+import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -30,22 +31,27 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Testcase for HBASE-23636.
  */
 @Category({ MasterTests.class, LargeTests.class })
 public class TestRaceBetweenSCPAndDTP {
+  private static final Logger LOG = LoggerFactory.getLogger(TestRaceBetweenSCPAndDTP.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -113,22 +119,26 @@ public class TestRaceBetweenSCPAndDTP {
     RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(NAME).get(0).getRegionInfo();
     AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
     ServerName sn = am.getRegionStates().getRegionState(region).getServerName();
+    LOG.info("ServerName={}, region={}", sn, region);
 
     ARRIVE_GET_REGIONS_ON_TABLE = new CountDownLatch(1);
     RESUME_GET_REGIONS_ON_SERVER = new CountDownLatch(1);
-
+    // Assign to local variable because this static gets set to null in above running thread and
+    // so NPE.
+    CountDownLatch cdl = ARRIVE_GET_REGIONS_ON_TABLE;
     UTIL.getAdmin().disableTableAsync(NAME);
-    ARRIVE_GET_REGIONS_ON_TABLE.await();
+    cdl.await();
 
-    UTIL.getMiniHBaseCluster().stopRegionServer(sn);
-    // Wait ServerCrashProcedure to init.
-    Thread.sleep(1000);
     ProcedureExecutor<?> procExec =
-        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
-    long scpProcId =
-        procExec.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
-            .map(p -> (ServerCrashProcedure) p).findAny().get().getProcId();
-    UTIL.waitFor(60000, () -> procExec.isFinished(scpProcId));
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    UTIL.getMiniHBaseCluster().stopRegionServer(sn);
+    long pid = Procedure.NO_PROC_ID;
+    do {
+      Threads.sleep(1);
+      pid = getSCPPID(procExec);
+    } while (pid != Procedure.NO_PROC_ID);
+    final long scppid = pid;
+    UTIL.waitFor(60000, () -> procExec.isFinished(scppid));
     RESUME_GET_REGIONS_ON_SERVER.countDown();
 
     long dtpProcId =
@@ -136,4 +146,13 @@ public class TestRaceBetweenSCPAndDTP {
             .map(p -> (DisableTableProcedure) p).findAny().get().getProcId();
     UTIL.waitFor(60000, () -> procExec.isFinished(dtpProcId));
   }
+
+  /**
+   * @return Returns {@link Procedure#NO_PROC_ID} if no SCP found else actual pid.
+   */
+  private long getSCPPID(ProcedureExecutor<?> e) {
+    Optional<ServerCrashProcedure> optional = e.getProcedures().stream().
+      filter(p -> p instanceof ServerCrashProcedure).map(p -> (ServerCrashProcedure) p).findAny();
+    return optional.isPresent()? optional.get().getProcId(): Procedure.NO_PROC_ID;
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java
index 6286104..a1b25e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java
@@ -133,16 +133,20 @@ public class TestRaceBetweenSCPAndTRSP {
     AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
     ServerName sn = am.getRegionStates().getRegionState(region).getServerName();
 
+    // Assign the CountDownLatches that get nulled in background threads else we NPE checking
+    // the static.
     ARRIVE_REGION_OPENING = new CountDownLatch(1);
+    CountDownLatch arriveRegionOpening = ARRIVE_REGION_OPENING;
     RESUME_REGION_OPENING = new CountDownLatch(1);
     ARRIVE_GET_REGIONS_ON_SERVER = new CountDownLatch(1);
+    CountDownLatch arriveGetRegionsOnServer = ARRIVE_GET_REGIONS_ON_SERVER;
     RESUME_GET_REGIONS_ON_SERVER = new CountDownLatch(1);
 
     Future<byte[]> moveFuture = am.moveAsync(new RegionPlan(region, sn, sn));
-    ARRIVE_REGION_OPENING.await();
+    arriveRegionOpening.await();
 
     UTIL.getMiniHBaseCluster().killRegionServer(sn);
-    ARRIVE_GET_REGIONS_ON_SERVER.await();
+    arriveGetRegionsOnServer.await();
     RESUME_REGION_OPENING.countDown();
 
     moveFuture.get();