tillrohrmann commented on a change in pull request #14686:
URL: https://github.com/apache/flink/pull/14686#discussion_r559706352



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -270,9 +270,22 @@ private void restartTasksWithDelay(final 
FailureHandlingResult failureHandlingRe
         delayExecutor.schedule(
                 () ->
                         FutureUtils.assertNoException(
-                                cancelFuture.thenRunAsync(
-                                        restartTasks(executionVertexVersions, 
globalRecovery),
-                                        getMainThreadExecutor())),
+                                cancelFuture
+                                        .thenRunAsync(
+                                                restartTasks(
+                                                        
executionVertexVersions, globalRecovery),
+                                                getMainThreadExecutor())
+                                        .thenRunAsync(
+                                                () ->
+                                                        archiveExceptions(
+                                                                
failureHandlingResult.getError(),
+                                                                
executionVertexVersions.stream()
+                                                                        .map(
+                                                                               
 ExecutionVertexVersion
+                                                                               
         ::getExecutionVertexId)
+                                                                        
.collect(
+                                                                               
 Collectors
+                                                                               
         .toList())))),

Review comment:
       I think this change goes in a good direction. But what we need to do 
before resetting the `Executions` in `restartTasks` is to collect all failure 
from the set of reset/restarted `Executions` and group them together with the 
`rootCause` into a failure information object.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -270,9 +270,22 @@ private void restartTasksWithDelay(final 
FailureHandlingResult failureHandlingRe
         delayExecutor.schedule(
                 () ->
                         FutureUtils.assertNoException(
-                                cancelFuture.thenRunAsync(
-                                        restartTasks(executionVertexVersions, 
globalRecovery),
-                                        getMainThreadExecutor())),
+                                cancelFuture
+                                        .thenRunAsync(
+                                                restartTasks(
+                                                        
executionVertexVersions, globalRecovery),
+                                                getMainThreadExecutor())
+                                        .thenRunAsync(
+                                                () ->
+                                                        archiveExceptions(
+                                                                
failureHandlingResult.getError(),
+                                                                
executionVertexVersions.stream()
+                                                                        .map(
+                                                                               
 ExecutionVertexVersion
+                                                                               
         ::getExecutionVertexId)
+                                                                        
.collect(
+                                                                               
 Collectors
+                                                                               
         .toList())))),

Review comment:
       There is also a list of prior `Executions` in 
`ExecutionVertex.priorExecutions`.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
##########
@@ -270,9 +270,22 @@ private void restartTasksWithDelay(final 
FailureHandlingResult failureHandlingRe
         delayExecutor.schedule(
                 () ->
                         FutureUtils.assertNoException(
-                                cancelFuture.thenRunAsync(
-                                        restartTasks(executionVertexVersions, 
globalRecovery),
-                                        getMainThreadExecutor())),
+                                cancelFuture
+                                        .thenRunAsync(
+                                                restartTasks(
+                                                        
executionVertexVersions, globalRecovery),
+                                                getMainThreadExecutor())
+                                        .thenRunAsync(
+                                                () ->
+                                                        archiveExceptions(
+                                                                
failureHandlingResult.getError(),
+                                                                
executionVertexVersions.stream()
+                                                                        .map(
+                                                                               
 ExecutionVertexVersion
+                                                                               
         ::getExecutionVertexId)
+                                                                        
.collect(
+                                                                               
 Collectors
+                                                                               
         .toList())))),

Review comment:
       I think in some way you are already doing it with the `exceptionCache`. 
It might be easier to read the `Execution.failureCause` for this.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to