ThreadPoolTaskRunner: Make graceful shutdown logs less scary.

- It's okay to suppress InterruptedException during graceful shutdown, as
  tasks may use it to accelerate their own shutdown.
- It's okay to ignore return statuses during graceful shutdown (which may
  be FAILED!) because it actually doesn't matter what they are.
This commit is contained in:
Gian Merlino 2016-03-10 19:38:26 -08:00
parent 1e49092ce7
commit 05397a9b4f

View File

@ -21,7 +21,6 @@ package io.druid.indexing.overlord;
import com.google.common.base.Optional; import com.google.common.base.Optional;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.FutureCallback;
@ -84,6 +83,8 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
private final TaskLocation location; private final TaskLocation location;
private volatile boolean stopping = false;
@Inject @Inject
public ThreadPoolTaskRunner( public ThreadPoolTaskRunner(
TaskToolboxFactory toolboxFactory, TaskToolboxFactory toolboxFactory,
@ -132,6 +133,8 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
@LifecycleStop @LifecycleStop
public void stop() public void stop()
{ {
stopping = true;
for (Map.Entry<Integer, ListeningExecutorService> entry : exec.entrySet()) { for (Map.Entry<Integer, ListeningExecutorService> entry : exec.entrySet()) {
try { try {
entry.getValue().shutdown(); entry.getValue().shutdown();
@ -159,11 +162,12 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
new Interval(new DateTime(start), taskConfig.getGracefulShutdownTimeout()).toDurationMillis(), new Interval(new DateTime(start), taskConfig.getGracefulShutdownTimeout()).toDurationMillis(),
TimeUnit.MILLISECONDS TimeUnit.MILLISECONDS
); );
// Ignore status, it doesn't matter for graceful shutdowns.
log.info( log.info(
"Graceful shutdown of task[%s] finished in %,dms with status[%s].", "Graceful shutdown of task[%s] finished in %,dms.",
task.getId(), task.getId(),
System.currentTimeMillis() - start, System.currentTimeMillis() - start
taskStatus.getStatusCode()
); );
} }
catch (Exception e) { catch (Exception e) {
@ -407,8 +411,16 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
status = task.run(toolbox); status = task.run(toolbox);
} }
catch (InterruptedException e) { catch (InterruptedException e) {
log.error(e, "Interrupted while running task[%s]", task); // Don't reset the interrupt flag of the thread, as we do want to continue to the end of this callable.
throw Throwables.propagate(e); if (stopping) {
// Tasks may interrupt their own run threads to stop themselves gracefully; don't be too scary about this.
log.debug(e, "Interrupted while running task[%s] during graceful shutdown.", task);
} else {
// Not stopping, this is definitely unexpected.
log.warn(e, "Interrupted while running task[%s]", task);
}
status = TaskStatus.failure(task.getId());
} }
catch (Exception e) { catch (Exception e) {
log.error(e, "Exception while running task[%s]", task); log.error(e, "Exception while running task[%s]", task);
@ -416,16 +428,10 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
} }
catch (Throwable t) { catch (Throwable t) {
log.error(t, "Uncaught Throwable while running task[%s]", task); log.error(t, "Uncaught Throwable while running task[%s]", task);
throw Throwables.propagate(t); throw t;
} }
try {
return status.withDuration(System.currentTimeMillis() - startTime); return status.withDuration(System.currentTimeMillis() - startTime);
} }
catch (Exception e) {
log.error(e, "Uncaught Exception during callback for task[%s]", task);
throw Throwables.propagate(e);
}
}
} }
} }