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

hbase git commit: HBASE-20169 NPE when calling HBTU.shutdownMiniCluster (TestAssignmentManagerMetrics is flakey); AMENDMENT

Repository: hbase
Updated Branches:
  refs/heads/branch-2.0 e8cf29047 -> 877f6b3c7


HBASE-20169 NPE when calling HBTU.shutdownMiniCluster (TestAssignmentManagerMetrics is flakey); AMENDMENT


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

Branch: refs/heads/branch-2.0
Commit: 877f6b3c753242d3e085bef15fb8927ea356ae64
Parents: e8cf290
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Tue May 1 15:16:13 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed May 2 16:14:11 2018 -0700

----------------------------------------------------------------------
 .../hbase/procedure2/ProcedureExecutor.java     | 41 ++++++++++++++------
 .../master/TestAssignmentManagerMetrics.java    |  6 +++
 2 files changed, 36 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/877f6b3c/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index e5105a5..3a75d33 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -264,9 +264,31 @@ public class ProcedureExecutor<TEnvironment> {
   private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners = new CopyOnWriteArrayList<>();
 
   private Configuration conf;
+
+  /**
+   * Created in the {@link #start(int, boolean)} method. Destroyed in {@link #join()} (FIX! Doing
+   * resource handling rather than observing in a #join is unexpected).
+   * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
+   * (Should be ok).
+   */
   private ThreadGroup threadGroup;
+
+  /**
+   * Created in the {@link #start(int, boolean)} method. Terminated in {@link #join()} (FIX! Doing
+   * resource handling rather than observing in a #join is unexpected).
+   * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
+   * (Should be ok).
+   */
   private CopyOnWriteArrayList<WorkerThread> workerThreads;
+
+  /**
+   * Created in the {@link #start(int, boolean)} method. Terminated in {@link #join()} (FIX! Doing
+   * resource handling rather than observing in a #join is unexpected).
+   * Overridden when we do the ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery
+   * (Should be ok).
+   */
   private TimeoutExecutorThread timeoutExecutor;
+
   private int corePoolSize;
   private int maxPoolSize;
 
@@ -299,6 +321,7 @@ public class ProcedureExecutor<TEnvironment> {
     this.conf = conf;
     this.checkOwnerSet = conf.getBoolean(CHECK_OWNER_SET_CONF_KEY, DEFAULT_CHECK_OWNER_SET);
     refreshConfiguration(conf);
+
   }
 
   private void load(final boolean abortOnCorruption) throws IOException {
@@ -510,11 +533,8 @@ public class ProcedureExecutor<TEnvironment> {
     LOG.info("Starting {} core workers (bigger of cpus/4 or 16) with max (burst) worker count={}",
         corePoolSize, maxPoolSize);
 
-    // Create the Thread Group for the executors
-    threadGroup = new ThreadGroup("PEWorkerGroup");
-
-    // Create the timeout executor
-    timeoutExecutor = new TimeoutExecutorThread(this, threadGroup);
+    this.threadGroup = new ThreadGroup("PEWorkerGroup");
+    this.timeoutExecutor = new TimeoutExecutorThread(this, threadGroup);
 
     // Create the workers
     workerId.set(0);
@@ -576,22 +596,21 @@ public class ProcedureExecutor<TEnvironment> {
 
     // stop the timeout executor
     timeoutExecutor.awaitTermination();
-    timeoutExecutor = null;
 
     // stop the worker threads
     for (WorkerThread worker: workerThreads) {
       worker.awaitTermination();
     }
-    workerThreads = null;
 
     // Destroy the Thread Group for the executors
+    // TODO: Fix. #join is not place to destroy resources.
     try {
       threadGroup.destroy();
     } catch (IllegalThreadStateException e) {
-      LOG.error("ThreadGroup " + threadGroup + " contains running threads; " + e.getMessage());
-      threadGroup.list();
-    } finally {
-      threadGroup = null;
+      LOG.error("ThreadGroup {} contains running threads; {}: See STDOUT",
+          this.threadGroup, e.getMessage());
+      // This dumps list of threads on STDOUT.
+      this.threadGroup.list();
     }
 
     // reset the in-memory state for testing

http://git-wip-us.apache.org/repos/asf/hbase/blob/877f6b3c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
index 87f6fa4..4c9d0e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
@@ -33,6 +33,7 @@ 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.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -91,6 +92,8 @@ public class TestAssignmentManagerMetrics {
     // set a small interval for updating rit metrics
     conf.setInt(AssignmentManager.RIT_CHORE_INTERVAL_MSEC_CONF_KEY, MSG_INTERVAL);
 
+    // keep rs online so it can report the failed opens.
+    conf.setBoolean(CoprocessorHost.ABORT_ON_ERROR_KEY, false);
     TEST_UTIL.startMiniCluster(1);
     CLUSTER = TEST_UTIL.getHBaseCluster();
     MASTER = CLUSTER.getMaster();
@@ -148,6 +151,9 @@ public class TestAssignmentManagerMetrics {
       }
 
       // Sleep 3 seconds, wait for doMetrics chore catching up
+      // the rit count consists of rit and failed opens. see RegionInTransitionStat#update
+      // Waiting for the completion of rit makes the assert stable.
+      TEST_UTIL.waitUntilNoRegionsInTransition();
       Thread.sleep(MSG_INTERVAL * 3);
       METRICS_HELPER.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_NAME, 1, amSource);
       METRICS_HELPER.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_OVER_THRESHOLD_NAME, 1,