-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Kill all running tasks when the supervisor task is killed #7041
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| package org.apache.druid.indexing.common.task.batch.parallel; | ||
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.util.concurrent.Futures; | ||
| import com.google.common.util.concurrent.ListenableFuture; | ||
|
|
@@ -84,6 +85,11 @@ public class TaskMonitor<T extends Task> | |
| private int numRunningTasks; | ||
| private int numSucceededTasks; | ||
| private int numFailedTasks; | ||
| // This metric is used only for unit tests because the current taskStatus system doesn't track the killed task status. | ||
| // Currently, this metric only represents # of killed tasks by ParallelIndexTaskRunner. | ||
| // See killAllRunningTasks(), SinglePhaseParallelIndexTaskRunner.run(), and | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was no method
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oops, thanks. Will fix it soon.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| // SinglePhaseParallelIndexTaskRunner.stopGracefully() | ||
| private int numKilledTasks; | ||
|
|
||
| private boolean running = false; | ||
|
|
||
|
|
@@ -169,11 +175,35 @@ public void start(long taskStatusCheckingPeriod) | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Stop task monitoring and kill all running tasks. | ||
| */ | ||
| public void stop() | ||
| { | ||
| synchronized (startStopLock) { | ||
| running = false; | ||
| taskStatusChecker.shutdownNow(); | ||
|
|
||
| if (numRunningTasks > 0) { | ||
| final Iterator<MonitorEntry> iterator = runningTasks.values().iterator(); | ||
| while (iterator.hasNext()) { | ||
| final MonitorEntry entry = iterator.next(); | ||
| iterator.remove(); | ||
| final String taskId = entry.runningTask.getId(); | ||
| log.info("Request to kill subtask[%s]", taskId); | ||
| indexingServiceClient.killTask(taskId); | ||
| numRunningTasks--; | ||
| numKilledTasks++; | ||
| } | ||
|
|
||
| if (numRunningTasks > 0) { | ||
| log.warn( | ||
| "Inconsistent state: numRunningTasks[%d] is still not zero after trying to kill all running tasks.", | ||
| numRunningTasks | ||
| ); | ||
| } | ||
| } | ||
|
|
||
| log.info("Stopped taskMonitor"); | ||
| } | ||
| } | ||
|
|
@@ -225,27 +255,14 @@ private void retry(String subTaskSpecId, MonitorEntry monitorEntry, TaskStatusPl | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * This method should be called after {@link #stop()} to make sure no additional tasks are submitted. | ||
| */ | ||
| void killAll() | ||
| { | ||
| runningTasks.values().forEach(entry -> { | ||
| final String taskId = entry.runningTask.getId(); | ||
| log.info("Request to kill subtask[%s]", taskId); | ||
| indexingServiceClient.killTask(taskId); | ||
| }); | ||
| runningTasks.clear(); | ||
| } | ||
|
|
||
| void incrementNumRunningTasks() | ||
| private void incrementNumRunningTasks() | ||
| { | ||
| synchronized (taskCountLock) { | ||
| numRunningTasks++; | ||
| } | ||
| } | ||
|
|
||
| void incrementNumSucceededTasks() | ||
| private void incrementNumSucceededTasks() | ||
| { | ||
| synchronized (taskCountLock) { | ||
| numRunningTasks--; | ||
|
|
@@ -254,7 +271,7 @@ void incrementNumSucceededTasks() | |
| } | ||
| } | ||
|
|
||
| void incrementNumFailedTasks() | ||
| private void incrementNumFailedTasks() | ||
| { | ||
| synchronized (taskCountLock) { | ||
| numRunningTasks--; | ||
|
|
@@ -276,6 +293,12 @@ int getNumRunningTasks() | |
| } | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| int getNumKilledTasks() | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. May make sense to add the annotation |
||
| { | ||
| return numKilledTasks; | ||
| } | ||
|
|
||
| SinglePhaseParallelIndexingProgress getProgress() | ||
| { | ||
| synchronized (taskCountLock) { | ||
|
|
@@ -336,7 +359,7 @@ class MonitorEntry | |
| @Nullable | ||
| private volatile TaskStatusPlus runningStatus; | ||
|
|
||
| MonitorEntry( | ||
| private MonitorEntry( | ||
| SubTaskSpec<T> spec, | ||
| T runningTask, | ||
| @Nullable TaskStatusPlus runningStatus, | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
wondering what if
getNumSplits()returns 1There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If it's 1, the native parallel task currently does the same thing: the supervisor task will run a sub task for the single split. Do you think it's better to process in the supervisor task rather than processing in the sub task?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
may be i was thinking in that direction, but I guess it's okay to run a sub task for single split.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. I think it makes sense, but probably better to fix in a separate PR.