tillrohrmann commented on a change in pull request #9072: [FLINK-11630] Wait
for the termination of all running Tasks when shutting down TaskExecutor
URL: https://github.com/apache/flink/pull/9072#discussion_r313897661
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
##########
@@ -333,32 +338,38 @@ private void
handleStartTaskExecutorServicesException(Exception e) throws Except
public CompletableFuture<Void> onStop() {
log.info("Stopping TaskExecutor {}.", getAddress());
- Throwable throwable = null;
+ Throwable jobManagerDisconnectThrowable = null;
if (resourceManagerConnection != null) {
resourceManagerConnection.close();
}
+ FlinkException cause = new FlinkException("The TaskExecutor is
shutting down.");
for (JobManagerConnection jobManagerConnection :
jobManagerConnections.values()) {
try {
-
disassociateFromJobManager(jobManagerConnection, new FlinkException("The
TaskExecutor is shutting down."));
+
disassociateFromJobManager(jobManagerConnection, cause);
} catch (Throwable t) {
- throwable = ExceptionUtils.firstOrSuppressed(t,
throwable);
+ jobManagerDisconnectThrowable =
ExceptionUtils.firstOrSuppressed(t, jobManagerDisconnectThrowable);
}
}
- try {
- stopTaskExecutorServices();
- } catch (Exception e) {
- throwable = ExceptionUtils.firstOrSuppressed(e,
throwable);
- }
-
- if (throwable != null) {
- return FutureUtils.completedExceptionally(new
FlinkException("Error while shutting the TaskExecutor down.", throwable));
- } else {
- log.info("Stopped TaskExecutor {}.", getAddress());
- return CompletableFuture.completedFuture(null);
- }
+ final Throwable throwableBeforeTasksCompletion =
jobManagerDisconnectThrowable;
+ return taskCompletionTracker
Review comment:
```suggestion
final CompletableFuture<Void> taskTerminationFuture =
taskCompletionTracker.failIncompleteTasksAndGetTerminationFuture();
return FutureUtils
.runAfterwards(
taskTerminationFuture,
() -> stopTaskExecutorServices())
.handle(
(ignored, throwable) -> {
if (throwable != null) {
throw new CompletionException(new
FlinkException("Error while shutting the TaskExecutor down.", throwable));
} else {
log.info("Stopped TaskExecutor {}.",
getAddress());
return null;
}
});
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services