azagrebin commented on a change in pull request #7757: [FLINK-11630] Triggers 
the termination of all running Tasks when shutting down TaskExecutor
URL: https://github.com/apache/flink/pull/7757#discussion_r294376336
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -347,18 +352,39 @@ private void 
handleStartTaskExecutorServicesException(Exception e) throws Except
 
                resourceManagerHeartbeatManager.stop();
 
-               try {
-                       stopTaskExecutorServices();
-               } catch (Exception e) {
-                       throwable = ExceptionUtils.firstOrSuppressed(e, 
throwable);
-               }
+               List<CompletableFuture<Void>> taskCompletionFutures = 
failSubmittedTasks();
 
                if (throwable != null) {
+                       try {
+                               stopTaskExecutorServices();
+                       } catch (Exception e) {
+                               throwable = ExceptionUtils.firstOrSuppressed(e, 
throwable);
+                       }
                        return FutureUtils.completedExceptionally(new 
FlinkException("Error while shutting the TaskExecutor down.", throwable));
                } else {
-                       log.info("Stopped TaskExecutor {}.", getAddress());
-                       return CompletableFuture.completedFuture(null);
+                       return 
FutureUtils.waitForAll(taskCompletionFutures).thenRun(() -> {
+                               try {
+                                       stopTaskExecutorServices();
+                               } catch (Exception e) {
+                                       throw new FlinkRuntimeException("Error 
while shutting the TaskExecutor down.", e);
+                               }
+                               log.info("Stopped TaskExecutor {}.", 
getAddress());
+                       });
+               }
+       }
+
+       private List<CompletableFuture<Void>> failSubmittedTasks() {
+               List<CompletableFuture<Void>> taskCompletionFutures = new 
ArrayList<>(submittedTasks.size());
+               FlinkException cause = new FlinkException("The TaskExecutor is 
shutting down.");
+               for (Task task : submittedTasks) {
+                       try {
+                               task.failExternally(cause);
+                               
taskCompletionFutures.add(task.getTaskCompletionFuture());
+                       } catch (IllegalStateException e) {
 
 Review comment:
   `IllegalStateException` should not happen at the moment, it would be an 
implementation error and the method should fail fast without catch then. We 
could optionally check:
   ```
   if (!task.getExecutionState().isTerminal()) {
     task.failExternally(cause);
   }
   ```

----------------------------------------------------------------
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

Reply via email to