Skip to content

Commit

Permalink
@HBASE-26468 Region Server doesn't exit cleanly incase it crashes.
Browse files Browse the repository at this point in the history
  • Loading branch information
shahrs87 committed Nov 30, 2021
1 parent 13787ae commit ac26674
Show file tree
Hide file tree
Showing 2 changed files with 52 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,24 +25,25 @@
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 =
Expand Down Expand Up @@ -307,4 +308,33 @@ public static synchronized void printThreadInfo(PrintStream stream, String title
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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
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.
Expand Down Expand Up @@ -119,15 +120,31 @@ public static void logProcessInfo(Configuration conf) {
}

/**
* 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 {
int ret = ToolRunner.run(HBaseConfiguration.create(), this, args);
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);
Expand Down

0 comments on commit ac26674

Please sign in to comment.