metaswirl commented on a change in pull request #18689:
URL: https://github.com/apache/flink/pull/18689#discussion_r803453533



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StateWithExecutionGraph.java
##########
@@ -148,6 +162,33 @@ public Logger getLogger() {
         return logger;
     }
 
+    protected Throwable extractError(TaskExecutionStateTransition 
taskExecutionStateTransition) {
+        Throwable cause = 
taskExecutionStateTransition.getError(userCodeClassLoader);
+        if (cause == null) {
+            cause = new FlinkException("Unknown failure cause. Probably 
related to FLINK-21376.");
+        }
+        return cause;
+    }
+
+    protected Optional<ExecutionVertexID> extractExecutionVertexID(
+            TaskExecutionStateTransition taskExecutionStateTransition) {
+        return 
executionGraph.getExecutionVertexId(taskExecutionStateTransition.getID());
+    }
+
+    protected static Optional<RootExceptionHistoryEntry> convertFailures(
+            Function<ExecutionVertexID, Optional<ExecutionVertex>> lookup,
+            List<Failure> failureCollection) {
+        if (failureCollection.isEmpty()) {
+            return Optional.empty();
+        }
+        Failure first = failureCollection.remove(0);
+        Set<ExceptionHistoryEntry> entries = new HashSet<>();
+        for (Failure failure : failureCollection) {
+            entries.add(failure.toExceptionHistoryEntry(lookup));

Review comment:
       Explanation:
   **Filter redundant failures**
   This can only happen, when a task is updated twice to failed with the same 
error. Or when the same global failure is reported twice.
   
   **Scan all Executions**
   Something like this:
   `executions.filter(execution -> 
execution.getFailureInfo().isPresent()).collect(Collectors.toSet())`
   
   This would identify executions that have failed, but not reported (or not in 
time) via `updateTaskExecutionState`.




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to