You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2021/12/01 05:49:43 UTC

[hbase] branch branch-1 updated: HBASE-26468 Region Server doesn't exit cleanly incase it crashes. (#3866) (#3862)

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

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


The following commit(s) were added to refs/heads/branch-1 by this push:
     new 7192423  HBASE-26468 Region Server doesn't exit cleanly incase it crashes. (#3866) (#3862)
7192423 is described below

commit 7192423ae7bcda12103aa6ece00578f49a5e3253
Author: Rushabh Shah <sh...@gmail.com>
AuthorDate: Wed Dec 1 00:49:13 2021 -0500

    HBASE-26468 Region Server doesn't exit cleanly incase it crashes. (#3866) (#3862)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Geoffrey Jacoby <gj...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../java/org/apache/hadoop/hbase/util/Threads.java | 36 ++++++++++++++++++++--
 .../hadoop/hbase/util/ServerCommandLine.java       | 21 +++++++++++--
 2 files changed, 52 insertions(+), 5 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
index fa02b25..8f42ae9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
@@ -25,24 +25,25 @@ import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.charset.StandardCharsets;
+import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Thread Utility
  */
 @InterfaceAudience.Private
 public class Threads {
-  private static final Log LOG = LogFactory.getLog(Threads.class);
+  private static final Logger LOG = LoggerFactory.getLogger(Threads.class);
   private static final AtomicInteger poolNumber = new AtomicInteger(1);
 
   public static final UncaughtExceptionHandler LOGGING_EXCEPTION_HANDLER =
@@ -307,4 +308,33 @@ public class Threads {
       throw new RuntimeException(e.getCause());
     }
   }
+
+  /**
+   * Checks whether any non-daemon thread is running.
+   * @return true if there are non daemon threads running, otherwise false
+   */
+  public static boolean isNonDaemonThreadRunning() {
+    AtomicInteger nonDaemonThreadCount = new AtomicInteger();
+    Set<Thread> threads =  Thread.getAllStackTraces().keySet();
+    for (Thread t: threads) {
+      // Exclude current thread
+      if (t.getId() != Thread.currentThread().getId() && !t.isDaemon()) {
+        nonDaemonThreadCount.getAndIncrement();
+        LOG.info("Non daemon thread {} is still alive", t.getName());
+        LOG.info(printStackTrace(t));
+      }
+    }
+    return nonDaemonThreadCount.get() > 0;
+  }
+
+  /*
+    Print stack trace of the passed thread
+   */
+  public static String printStackTrace(Thread t) {
+    StringBuilder sb = new StringBuilder();
+    for (StackTraceElement frame: t.getStackTrace()) {
+      sb.append("\n").append("    ").append(frame.toString());
+    }
+    return sb.toString();
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
index e6b746c..42f8722 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import static org.apache.hadoop.hbase.util.Threads.isNonDaemonThreadRunning;
 
 /**
  * Base class for command lines that start up various HBase daemons.
@@ -119,8 +120,10 @@ public abstract class ServerCommandLine extends Configured implements Tool {
   }
 
   /**
-   * Parse and run the given command line. This may exit the JVM if
-   * a nonzero exit code is returned from <code>run()</code>.
+   * Parse and run the given command line. This will exit the JVM with
+   * the exit code returned from <code>run()</code>.
+   * If return code is 0, wait for atmost 30 seconds for all non-daemon threads to quit,
+   * otherwise exit the jvm
    */
   public void doMain(String args[]) {
     try {
@@ -128,6 +131,20 @@ public abstract class ServerCommandLine extends Configured implements Tool {
       if (ret != 0) {
         System.exit(ret);
       }
+      // Return code is 0 here.
+      boolean forceStop = false;
+      long startTime = EnvironmentEdgeManager.currentTime();
+      while (isNonDaemonThreadRunning()) {
+        if (EnvironmentEdgeManager.currentTime() - startTime > 30 * 1000) {
+          forceStop = true;
+          break;
+        }
+        Thread.sleep(1000);
+      }
+      if (forceStop) {
+        LOG.error("Failed to stop all non-daemon threads, so terminating JVM");
+        System.exit(-1);
+      }
     } catch (Exception e) {
       LOG.error("Failed to run", e);
       System.exit(-1);