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 2020/03/30 23:06:33 UTC

[hbase] branch branch-2.3 updated: HBASE-24052 Add debug+fix to TestMasterShutdown (#1373)

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

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


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new ae30c9c  HBASE-24052 Add debug+fix to TestMasterShutdown (#1373)
ae30c9c is described below

commit ae30c9cab3b31cde9a251443af2b3ef4fb87bb5e
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Mon Mar 30 16:06:24 2020 -0700

    HBASE-24052 Add debug+fix to TestMasterShutdown (#1373)
    
    Add check for stopped server at a few more points in Master startup.
    Defend against NPE in RSProcedureDispatcher; log and retun instead.
    
    Signed-off-by: Bharath Vissapragada <bh...@apache.org>
---
 .../org/apache/hadoop/hbase/master/HMaster.java    |  1 +
 .../master/procedure/RSProcedureDispatcher.java    | 21 ++++++--
 .../hadoop/hbase/regionserver/HRegionServer.java   | 39 ++++++++-------
 .../hadoop/hbase/master/TestMasterShutdown.java    | 58 +++++-----------------
 4 files changed, 53 insertions(+), 66 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 970cda8..232b3ab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -2751,6 +2751,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   @Override
   public void abort(String reason, Throwable cause) {
     if (isAborted() || isStopped()) {
+      LOG.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());
       return;
     }
     setAbortRequested();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 10b823c..ea5a165 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.ServerListener;
 import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
 import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
@@ -95,10 +96,24 @@ public class RSProcedureDispatcher
       return false;
     }
     // Around startup, if failed, some of the below may be set back to null so NPE is possible.
+    ServerManager sm = master.getServerManager();
+    if (sm == null) {
+      LOG.debug("ServerManager is null; stopping={}", master.isStopping());
+      return false;
+    }
+    sm.registerListener(this);
+    ProcedureExecutor<MasterProcedureEnv> pe = master.getMasterProcedureExecutor();
+    if (pe == null) {
+      LOG.debug("ProcedureExecutor is null; stopping={}", master.isStopping());
+      return false;
+    }
+    procedureEnv = pe.getEnvironment();
+    if (this.procedureEnv == null) {
+      LOG.debug("ProcedureEnv is null; stopping={}", master.isStopping());
+      return false;
+    }
     try {
-      master.getServerManager().registerListener(this);
-      procedureEnv = master.getMasterProcedureExecutor().getEnvironment();
-      for (ServerName serverName : master.getServerManager().getOnlineServersList()) {
+      for (ServerName serverName : sm.getOnlineServersList()) {
         addNode(serverName);
       }
     } catch (Exception e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0de2c17..904447c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -951,6 +951,10 @@ public class HRegionServer extends HasThread implements
    */
   @Override
   public void run() {
+    if (isStopped()) {
+      LOG.info("Skipping run; stopped");
+      return;
+    }
     try {
       // Do pre-registration initializations; zookeeper, lease threads, etc.
       preRegistrationInitialization();
@@ -964,24 +968,25 @@ public class HRegionServer extends HasThread implements
         // Initialize the RegionServerCoprocessorHost now that our ephemeral
         // node was created, in case any coprocessors want to use ZooKeeper
         this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
-      }
 
-      // Try and register with the Master; tell it we are here.  Break if server is stopped or the
-      // clusterup flag is down or hdfs went wacky. Once registered successfully, go ahead and start
-      // up all Services. Use RetryCounter to get backoff in case Master is struggling to come up.
-      LOG.debug("About to register with Master.");
-      RetryCounterFactory rcf = new RetryCounterFactory(Integer.MAX_VALUE,
-          this.sleeper.getPeriod(), 1000 * 60 * 5);
-      RetryCounter rc = rcf.create();
-      while (keepLooping()) {
-        RegionServerStartupResponse w = reportForDuty();
-        if (w == null) {
-          long sleepTime = rc.getBackoffTimeAndIncrementAttempts();
-          LOG.warn("reportForDuty failed; sleeping {} ms and then retrying.", sleepTime);
-          this.sleeper.sleep(sleepTime);
-        } else {
-          handleReportForDutyResponse(w);
-          break;
+        // Try and register with the Master; tell it we are here.  Break if server is stopped or
+        // the clusterup flag is down or hdfs went wacky. Once registered successfully, go ahead and
+        // start up all Services. Use RetryCounter to get backoff in case Master is struggling to
+        // come up.
+        LOG.debug("About to register with Master.");
+        RetryCounterFactory rcf =
+          new RetryCounterFactory(Integer.MAX_VALUE, this.sleeper.getPeriod(), 1000 * 60 * 5);
+        RetryCounter rc = rcf.create();
+        while (keepLooping()) {
+          RegionServerStartupResponse w = reportForDuty();
+          if (w == null) {
+            long sleepTime = rc.getBackoffTimeAndIncrementAttempts();
+            LOG.warn("reportForDuty failed; sleeping {} ms and then retrying.", sleepTime);
+            this.sleeper.sleep(sleepTime);
+          } else {
+            handleReportForDutyResponse(w);
+            break;
+          }
         }
       }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
index a5e596f..7b3921e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
@@ -21,10 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import java.io.IOException;
-import java.time.Duration;
 import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics;
@@ -35,8 +32,6 @@ import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.AsyncConnection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@@ -156,48 +151,19 @@ public class TestMasterShutdown {
       hbaseCluster = new LocalHBaseCluster(htu.getConfiguration(), options.getNumMasters(),
         options.getNumRegionServers(), options.getMasterClass(), options.getRsClass());
       final MasterThread masterThread = hbaseCluster.getMasters().get(0);
-
-      final CompletableFuture<Void> shutdownFuture = CompletableFuture.runAsync(() -> {
-        // Switching to master registry exacerbated a race in the master bootstrap that can result
-        // in a lost shutdown command (HBASE-8422, HBASE-23836). The race is essentially because
-        // the server manager in HMaster is not initialized by the time shutdown() RPC (below) is
-        // made to the master. The suspected reason as to why it was uncommon before HBASE-18095
-        // is because the connection creation with ZK registry is so slow that by then the server
-        // manager is usually init'ed in time for the RPC to be made. For now, adding an explicit
-        // wait() in the test, waiting for the server manager to become available.
-        final long timeout = TimeUnit.MINUTES.toMillis(10);
-        assertNotEquals("timeout waiting for server manager to become available.",
-          -1, Waiter.waitFor(htu.getConfiguration(), timeout,
-            () -> masterThread.getMaster().getServerManager() != null));
-
-        // Master has come up far enough that we can terminate it without creating a zombie.
-        final long result = Waiter.waitFor(htu.getConfiguration(), timeout, 500, () -> {
-          final Configuration conf = createResponsiveZkConfig(htu.getConfiguration());
-          LOG.debug("Attempting to establish connection.");
-          final CompletableFuture<AsyncConnection> connFuture =
-            ConnectionFactory.createAsyncConnection(conf);
-          try (final AsyncConnection conn = connFuture.join()) {
-            LOG.debug("Sending shutdown RPC.");
-            try {
-              conn.getAdmin().shutdown().join();
-              LOG.debug("Shutdown RPC sent.");
-              return true;
-            } catch (CompletionException e) {
-              LOG.debug("Failure sending shutdown RPC.");
-            }
-          } catch (IOException|CompletionException e) {
-            LOG.debug("Failed to establish connection.");
-          } catch (Throwable e) {
-            LOG.info("Something unexpected happened.", e);
-          }
-          return false;
-        });
-        assertNotEquals("Failed to issue shutdown RPC after " + Duration.ofMillis(timeout),
-          -1, result);
-      });
-
       masterThread.start();
-      shutdownFuture.join();
+      // Switching to master registry exacerbated a race in the master bootstrap that can result
+      // in a lost shutdown command (HBASE-8422, HBASE-23836). The race is essentially because
+      // the server manager in HMaster is not initialized by the time shutdown() RPC (below) is
+      // made to the master. The suspected reason as to why it was uncommon before HBASE-18095
+      // is because the connection creation with ZK registry is so slow that by then the server
+      // manager is usually init'ed in time for the RPC to be made. For now, adding an explicit
+      // wait() in the test, waiting for the server manager to become available.
+      final long timeout = TimeUnit.MINUTES.toMillis(10);
+      assertNotEquals("Timeout waiting for server manager to become available.",
+        -1, Waiter.waitFor(htu.getConfiguration(), timeout,
+          () -> masterThread.getMaster().getServerManager() != null));
+      htu.getConnection().getAdmin().shutdown();
       masterThread.join();
     } finally {
       if (hbaseCluster != null) {