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:35:13 UTC

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

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

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


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

commit d1762f5209efd39044fafd9a2b85e31b8083f17e
Author: Rushabh Shah <sh...@gmail.com>
AuthorDate: Wed Dec 1 00:34:35 2021 -0500

    HBASE-26468 Region Server doesn't exit cleanly incase it crashes. (#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 | 31 ++++++++++++++++++++++
 .../hadoop/hbase/util/ServerCommandLine.java       | 21 +++++++++++++--
 2 files changed, 50 insertions(+), 2 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 610fe05..d80639e 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
@@ -20,10 +20,12 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.PrintStream;
 import java.lang.Thread.UncaughtExceptionHandler;
+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.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -197,4 +199,33 @@ public class Threads {
   public static void printThreadInfo(PrintStream stream, String title) {
     ReflectionUtils.printThreadInfo(stream, title);
   }
+
+  /**
+   * 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();
+    threads.forEach(t -> {
+      // 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 f99a090..7f89d59 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
@@ -36,6 +36,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import static org.apache.hadoop.hbase.util.Threads.isNonDaemonThreadRunning;
 
 /**
  * Base class for command lines that start up various HBase daemons.
@@ -141,8 +142,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 {
@@ -150,6 +153,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);