HADOOP-16181. HadoopExecutors shutdown Cleanup.

Author:    David Mollitor <david.mollitor@cloudera.com>
This commit is contained in:
David Mollitor 2019-03-22 10:29:27 +00:00 committed by Steve Loughran
parent 246ab77f28
commit d18d0859eb
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
1 changed files with 38 additions and 20 deletions

View File

@ -92,33 +92,51 @@ public final class HadoopExecutors {
}
/**
* Helper routine to shutdown a executorService.
* Helper routine to shutdown a {@link ExecutorService}. Will wait up to a
* certain timeout for the ExecutorService to gracefully shutdown. If the
* ExecutorService did not shutdown and there are still tasks unfinished after
* the timeout period, the ExecutorService will be notified to forcibly shut
* down. Another timeout period will be waited before giving up. So, at most,
* a shutdown will be allowed to wait up to twice the timeout value before
* giving up.
*
* @param executorService - executorService
* @param logger - Logger
* @param timeout - Timeout
* @param unit - TimeUnits, generally seconds.
* @param executorService ExecutorService to shutdown
* @param logger Logger
* @param timeout the maximum time to wait
* @param unit the time unit of the timeout argument
*/
public static void shutdown(ExecutorService executorService, Logger logger,
long timeout, TimeUnit unit) {
try {
if (executorService != null) {
executorService.shutdown();
try {
if (!executorService.awaitTermination(timeout, unit)) {
executorService.shutdownNow();
}
if (!executorService.awaitTermination(timeout, unit)) {
logger.error("Unable to shutdown properly.");
}
} catch (InterruptedException e) {
logger.error("Error attempting to shutdown.", e);
executorService.shutdownNow();
}
if (executorService == null) {
return;
}
try {
executorService.shutdown();
logger.info(
"Gracefully shutting down executor service. Waiting max {} {}",
timeout, unit);
if (!executorService.awaitTermination(timeout, unit)) {
logger.info(
"Executor service has not shutdown yet. Forcing. "
+ "Will wait up to an additional {} {} for shutdown",
timeout, unit);
executorService.shutdownNow();
}
if (executorService.awaitTermination(timeout, unit)) {
logger.info("Succesfully shutdown executor service");
} else {
logger.error("Unable to shutdown executor service after timeout {} {}",
(2 * timeout), unit);
}
} catch (InterruptedException e) {
logger.error("Interrupted while attempting to shutdown", e);
executorService.shutdownNow();
} catch (Exception e) {
logger.error("Error during shutdown: ", e);
logger.warn("Exception closing executor service {}", e.getMessage());
logger.debug("Exception closing executor service", e);
throw e;
}
}