Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: Geoffrey Jacoby <gjacoby@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
774484ed44
commit
6fc5bbc19a
|
@ -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;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -195,4 +197,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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue